You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2015/01/16 03:29:18 UTC

[01/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Repository: incubator-nifi
Updated Branches:
  refs/heads/develop ad74a4304 -> 300952a98


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
deleted file mode 100644
index 071be4d..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
+++ /dev/null
@@ -1,1243 +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;
-
-import org.apache.nifi.controller.ProcessScheduler;
-import org.apache.nifi.controller.ValidationContextFactory;
-import org.apache.nifi.controller.ProcessorNode;
-import static java.util.Objects.requireNonNull;
-
-import java.util.ArrayList;
-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.Objects;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-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.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.connectable.Connectable;
-import org.apache.nifi.connectable.ConnectableType;
-import org.apache.nifi.connectable.Connection;
-import org.apache.nifi.connectable.Position;
-import org.apache.nifi.controller.service.ControllerServiceProvider;
-import org.apache.nifi.groups.ProcessGroup;
-import org.apache.nifi.logging.LogLevel;
-import org.apache.nifi.logging.LogRepositoryFactory;
-import org.apache.nifi.nar.NarCloseable;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSessionFactory;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.processor.annotation.CapabilityDescription;
-import org.apache.nifi.processor.annotation.EventDriven;
-import org.apache.nifi.processor.annotation.SideEffectFree;
-import org.apache.nifi.processor.annotation.SupportsBatching;
-import org.apache.nifi.processor.annotation.TriggerSerially;
-import org.apache.nifi.processor.annotation.TriggerWhenAnyDestinationAvailable;
-import org.apache.nifi.processor.annotation.TriggerWhenEmpty;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.util.FormatUtils;
-
-import org.apache.commons.lang3.builder.EqualsBuilder;
-import org.apache.commons.lang3.builder.HashCodeBuilder;
-import org.quartz.CronExpression;
-import org.slf4j.LoggerFactory;
-
-/**
- * ProcessorNode provides thread-safe access to a FlowFileProcessor as it exists
- * within a controlled flow. This node keeps track of the processor, its
- * scheduling information and its relationships to other processors and whatever
- * scheduled futures exist for it. Must be thread safe.
- *
- * @author none
- */
-public class StandardProcessorNode extends ProcessorNode implements Connectable {
-
-    public static final String BULLETIN_OBSERVER_ID = "bulletin-observer";
-
-    public static final TimeUnit DEFAULT_TIME_UNIT = TimeUnit.MILLISECONDS;
-    public static final String DEFAULT_YIELD_PERIOD = "1 sec";
-    public static final String DEFAULT_PENALIZATION_PERIOD = "30 sec";
-    private final AtomicReference<ProcessGroup> processGroup;
-    private final Processor processor;
-    private final AtomicReference<String> identifier;
-    private final Map<Connection, Connectable> destinations;
-    private final Map<Relationship, Set<Connection>> connections;
-    private final AtomicReference<Set<Relationship>> undefinedRelationshipsToTerminate;
-    private final AtomicReference<List<Connection>> incomingConnectionsRef;
-    private final ReentrantReadWriteLock rwLock;
-    private final Lock readLock;
-    private final Lock writeLock;
-    private final AtomicBoolean isolated;
-    private final AtomicBoolean lossTolerant;
-    private final AtomicReference<ScheduledState> scheduledState;
-    private final AtomicReference<String> comments;
-    private final AtomicReference<String> name;
-    private final AtomicReference<Position> position;
-    private final AtomicReference<String> annotationData;
-    private final AtomicReference<String> schedulingPeriod; // stored as string so it's presented to user as they entered it
-    private final AtomicReference<String> yieldPeriod;
-    private final AtomicReference<String> penalizationPeriod;
-    private final AtomicReference<Map<String, String>> style;
-    private final AtomicInteger concurrentTaskCount;
-    private final AtomicLong yieldExpiration;
-    private final AtomicLong schedulingNanos;
-    private final boolean triggerWhenEmpty;
-    private final boolean sideEffectFree;
-    private final boolean triggeredSerially;
-    private final boolean triggerWhenAnyDestinationAvailable;
-    private final boolean eventDrivenSupported;
-    private final boolean batchSupported;
-    private final ValidationContextFactory validationContextFactory;
-    private final ProcessScheduler processScheduler;
-    private long runNanos = 0L;
-
-    private SchedulingStrategy schedulingStrategy;  // guarded by read/write lock
-
-    StandardProcessorNode(final Processor processor, final String uuid, final ValidationContextFactory validationContextFactory,
-            final ProcessScheduler scheduler, final ControllerServiceProvider controllerServiceProvider) {
-        super(processor, uuid, validationContextFactory, controllerServiceProvider);
-
-        this.processor = processor;
-        identifier = new AtomicReference<>(uuid);
-        destinations = new HashMap<>();
-        connections = new HashMap<>();
-        incomingConnectionsRef = new AtomicReference<List<Connection>>(new ArrayList<Connection>());
-        scheduledState = new AtomicReference<>(ScheduledState.STOPPED);
-        rwLock = new ReentrantReadWriteLock(false);
-        readLock = rwLock.readLock();
-        writeLock = rwLock.writeLock();
-        lossTolerant = new AtomicBoolean(false);
-        final Set<Relationship> emptySetOfRelationships = new HashSet<>();
-        undefinedRelationshipsToTerminate = new AtomicReference<>(emptySetOfRelationships);
-        comments = new AtomicReference<>("");
-        name = new AtomicReference<>(processor.getClass().getSimpleName());
-        schedulingPeriod = new AtomicReference<>("0 sec");
-        schedulingNanos = new AtomicLong(MINIMUM_SCHEDULING_NANOS);
-        yieldPeriod = new AtomicReference<>(DEFAULT_YIELD_PERIOD);
-        yieldExpiration = new AtomicLong(0L);
-        concurrentTaskCount = new AtomicInteger(1);
-        position = new AtomicReference<>(new Position(0D, 0D));
-        style = new AtomicReference<>(Collections.unmodifiableMap(new HashMap<String, String>()));
-        this.processGroup = new AtomicReference<>();
-        processScheduler = scheduler;
-        annotationData = new AtomicReference<>();
-        isolated = new AtomicBoolean(false);
-        penalizationPeriod = new AtomicReference<>(DEFAULT_PENALIZATION_PERIOD);
-
-        triggerWhenEmpty = processor.getClass().isAnnotationPresent(TriggerWhenEmpty.class);
-        sideEffectFree = processor.getClass().isAnnotationPresent(SideEffectFree.class);
-        batchSupported = processor.getClass().isAnnotationPresent(SupportsBatching.class);
-        triggeredSerially = processor.getClass().isAnnotationPresent(TriggerSerially.class);
-        triggerWhenAnyDestinationAvailable = processor.getClass().isAnnotationPresent(TriggerWhenAnyDestinationAvailable.class);
-        this.validationContextFactory = validationContextFactory;
-        eventDrivenSupported = processor.getClass().isAnnotationPresent(EventDriven.class) && !triggeredSerially && !triggerWhenEmpty;
-        schedulingStrategy = SchedulingStrategy.TIMER_DRIVEN;
-    }
-
-    /**
-     * @return comments about this specific processor instance
-     */
-    @Override
-    public String getComments() {
-        return comments.get();
-    }
-
-    /**
-     * Provides and opportunity to retain information about this particular
-     * processor instance
-     *
-     * @param comments
-     */
-    @Override
-    public void setComments(final String comments) {
-        writeLock.lock();
-        try {
-            if (isRunning()) {
-                throw new IllegalStateException("Cannot modify Processor configuration while the Processor is running");
-            }
-            this.comments.set(comments);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public ScheduledState getScheduledState() {
-        return scheduledState.get();
-    }
-
-    @Override
-    public Position getPosition() {
-        return position.get();
-    }
-
-    @Override
-    public void setPosition(Position position) {
-        this.position.set(position);
-    }
-
-    public Map<String, String> getStyle() {
-        return style.get();
-    }
-
-    public void setStyle(final Map<String, String> style) {
-        if (style != null) {
-            this.style.set(Collections.unmodifiableMap(new HashMap<>(style)));
-        }
-    }
-
-    @Override
-    public String getIdentifier() {
-        return identifier.get();
-    }
-
-    /**
-     * @return if true flow file content generated by this processor is
-     * considered loss tolerant
-     */
-    @Override
-    public boolean isLossTolerant() {
-        return lossTolerant.get();
-    }
-
-    /**
-     * @return if true processor runs only on the primary node
-     */
-    public boolean isIsolated() {
-        return isolated.get();
-    }
-
-    /**
-     * @return true if the processor has the {@link TriggerWhenEmpty}
-     * annotation, false otherwise.
-     */
-    @Override
-    public boolean isTriggerWhenEmpty() {
-        return triggerWhenEmpty;
-    }
-
-    /**
-     * @return true if the processor has the {@link SideEffectFree} annotation,
-     * false otherwise.
-     */
-    public boolean isSideEffectFree() {
-        return sideEffectFree;
-    }
-
-    @Override
-    public boolean isHighThroughputSupported() {
-        return batchSupported;
-    }
-
-    /**
-     * @return true if the processor has the
-     * {@link TriggerWhenAnyDestinationAvailable} annotation, false otherwise.
-     */
-    public boolean isTriggerWhenAnyDestinationAvailable() {
-        return triggerWhenAnyDestinationAvailable;
-    }
-
-    /**
-     * Indicates whether flow file content made by this processor must be
-     * persisted
-     *
-     * @param lossTolerant
-     */
-    @Override
-    public void setLossTolerant(final boolean lossTolerant) {
-        writeLock.lock();
-        try {
-            if (isRunning()) {
-                throw new IllegalStateException("Cannot modify Processor configuration while the Processor is running");
-            }
-            this.lossTolerant.set(lossTolerant);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * Indicates whether the processor runs on only the primary node.
-     *
-     * @param isolated
-     */
-    public void setIsolated(final boolean isolated) {
-        writeLock.lock();
-        try {
-            if (isRunning()) {
-                throw new IllegalStateException("Cannot modify Processor configuration while the Processor is running");
-            }
-            this.isolated.set(isolated);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public boolean isAutoTerminated(final Relationship relationship) {
-        final Set<Relationship> terminatable = undefinedRelationshipsToTerminate.get();
-        if (terminatable == null) {
-            return false;
-        }
-        return terminatable.contains(relationship);
-    }
-
-    /**
-     * Indicates whether flow files transferred to undefined relationships
-     * should be terminated
-     *
-     * @param terminate
-     */
-    @Override
-    public void setAutoTerminatedRelationships(final Set<Relationship> terminate) {
-        writeLock.lock();
-        try {
-            if (isRunning()) {
-                throw new IllegalStateException("Cannot modify Processor configuration while the Processor is running");
-            }
-
-            for (final Relationship rel : terminate) {
-                if (!getConnections(rel).isEmpty()) {
-                    throw new IllegalStateException("Cannot mark relationship '" + rel.getName() + "' as auto-terminated because Connection already exists with this relationship");
-                }
-            }
-            undefinedRelationshipsToTerminate.set(new HashSet<>(terminate));
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * @return an unmodifiable Set that contains all of the
-     * ProcessorRelationship objects that are configured to be auto-terminated
-     */
-    public Set<Relationship> getAutoTerminatedRelationships() {
-        Set<Relationship> relationships = undefinedRelationshipsToTerminate.get();
-        if (relationships == null) {
-            relationships = new HashSet<>();
-        }
-        return Collections.unmodifiableSet(relationships);
-    }
-
-    @Override
-    public String getName() {
-        return name.get();
-    }
-
-    /**
-     * @return the value of the processor's {@link CapabilityDescription}
-     * annotation, if one exists, else <code>null</code>.
-     */
-    public String getProcessorDescription() {
-        final CapabilityDescription capDesc = processor.getClass().getAnnotation(CapabilityDescription.class);
-        return (capDesc == null) ? null : capDesc.value();
-    }
-
-    @Override
-    public void setName(final String name) {
-        writeLock.lock();
-        try {
-            if (isRunning()) {
-                throw new IllegalStateException("Cannot modify Processor configuration while the Processor is running");
-            }
-            this.name.set(name);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * @param timeUnit determines the unit of time to represent the scheduling
-     * period. If null will be reported in units of
-     * {@link #DEFAULT_SCHEDULING_TIME_UNIT}
-     * @return the schedule period that should elapse before subsequent cycles
-     * of this processor's tasks
-     */
-    @Override
-    public long getSchedulingPeriod(final TimeUnit timeUnit) {
-        return timeUnit.convert(schedulingNanos.get(), TimeUnit.NANOSECONDS);
-    }
-
-    public boolean isEventDrivenSupported() {
-        readLock.lock();
-        try {
-            return this.eventDrivenSupported;
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    /**
-     * Updates the Scheduling Strategy used for this Processor
-     *
-     * @param schedulingStrategy
-     *
-     * @throws IllegalArgumentException if the SchedulingStrategy is not not
-     * applicable for this Processor
-     */
-    public void setSchedulingStrategy(final SchedulingStrategy schedulingStrategy) {
-        writeLock.lock();
-        try {
-            if (schedulingStrategy == SchedulingStrategy.EVENT_DRIVEN && !eventDrivenSupported) {
-                // not valid. Just ignore it. We don't throw an Exception because if a developer changes a Processor so that
-                // it no longer supports EventDriven mode, we don't want the app to fail to startup if it was already in Event-Driven
-                // Mode. Instead, we will simply leave it in Timer-Driven mode
-                return;
-            }
-
-            this.schedulingStrategy = schedulingStrategy;
-            setIsolated(schedulingStrategy == SchedulingStrategy.PRIMARY_NODE_ONLY);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * Returns the currently configured scheduling strategy
-     *
-     * @return
-     */
-    public SchedulingStrategy getSchedulingStrategy() {
-        readLock.lock();
-        try {
-            return this.schedulingStrategy;
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public String getSchedulingPeriod() {
-        return schedulingPeriod.get();
-    }
-
-    /**
-     * @param value the number of <code>timeUnit</code>s between scheduling
-     * intervals.
-     * @param timeUnit determines the unit of time to represent the scheduling
-     * period.
-     */
-    @Override
-    public void setScheduldingPeriod(final String schedulingPeriod) {
-        writeLock.lock();
-        try {
-            if (isRunning()) {
-                throw new IllegalStateException("Cannot modify Processor configuration while the Processor is running");
-            }
-
-            switch (schedulingStrategy) {
-                case CRON_DRIVEN: {
-                    try {
-                        new CronExpression(schedulingPeriod);
-                    } catch (final Exception e) {
-                        throw new IllegalArgumentException("Scheduling Period is not a valid cron expression: " + schedulingPeriod);
-                    }
-                }
-                break;
-                case PRIMARY_NODE_ONLY:
-                case TIMER_DRIVEN: {
-                    final long schedulingNanos = FormatUtils.getTimeDuration(requireNonNull(schedulingPeriod), TimeUnit.NANOSECONDS);
-                    if (schedulingNanos < 0) {
-                        throw new IllegalArgumentException("Scheduling Period must be positive");
-                    }
-                    this.schedulingNanos.set(Math.max(MINIMUM_SCHEDULING_NANOS, schedulingNanos));
-                }
-                break;
-                case EVENT_DRIVEN:
-                default:
-                    return;
-            }
-
-            this.schedulingPeriod.set(schedulingPeriod);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public long getRunDuration(final TimeUnit timeUnit) {
-        readLock.lock();
-        try {
-            return timeUnit.convert(this.runNanos, TimeUnit.NANOSECONDS);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public void setRunDuration(final long duration, final TimeUnit timeUnit) {
-        writeLock.lock();
-        try {
-            if (duration < 0) {
-                throw new IllegalArgumentException("Run Duration must be non-negative value; cannot set to " + timeUnit.toSeconds(duration) + " seconds");
-            }
-
-            this.runNanos = timeUnit.toNanos(duration);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * @param timeUnit determines the unit of time to represent the yield
-     * period. If null will be reported in units of
-     * {@link #DEFAULT_SCHEDULING_TIME_UNIT}.
-     * @return
-     */
-    @Override
-    public long getYieldPeriod(final TimeUnit timeUnit) {
-        return FormatUtils.getTimeDuration(getYieldPeriod(), timeUnit == null ? DEFAULT_TIME_UNIT : timeUnit);
-    }
-
-    public String getYieldPeriod() {
-        return yieldPeriod.get();
-    }
-
-    /**
-     * Updates the amount of time that this processor should avoid being
-     * scheduled when the processor calls
-     * {@link nifi.processor.ProcessContext#yield() ProcessContext.yield()}
-     *
-     * @param yieldPeriod
-     */
-    @Override
-    public void setYieldPeriod(final String yieldPeriod) {
-        writeLock.lock();
-        try {
-            if (isRunning()) {
-                throw new IllegalStateException("Cannot modify Processor configuration while the Processor is running");
-            }
-            final long yieldMillis = FormatUtils.getTimeDuration(requireNonNull(yieldPeriod), TimeUnit.MILLISECONDS);
-            if (yieldMillis < 0) {
-                throw new IllegalArgumentException("Yield duration must be positive");
-            }
-            this.yieldPeriod.set(yieldPeriod);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * Causes the processor not to be scheduled for some period of time. This
-     * duration can be obtained and set via the
-     * {@link #getYieldPeriod(TimeUnit)} and
-     * {@link #setYieldPeriod(long, TimeUnit)} methods.
-     */
-    @Override
-    public void yield() {
-        final long yieldMillis = getYieldPeriod(TimeUnit.MILLISECONDS);
-        yield(yieldMillis, TimeUnit.MILLISECONDS);
-
-        final String yieldDuration = (yieldMillis > 1000) ? (yieldMillis / 1000) + " seconds" : yieldMillis + " milliseconds";
-        LoggerFactory.getLogger(processor.getClass()).debug("{} has chosen to yield its resources; will not be scheduled to run again for {}", processor, yieldDuration);
-    }
-
-    public void yield(final long period, final TimeUnit timeUnit) {
-        final long yieldMillis = TimeUnit.MILLISECONDS.convert(period, timeUnit);
-        yieldExpiration.set(Math.max(yieldExpiration.get(), System.currentTimeMillis() + yieldMillis));
-
-        processScheduler.yield(this);
-    }
-
-    /**
-     * @return the number of milliseconds since Epoch at which time this
-     * processor is to once again be scheduled.
-     */
-    @Override
-    public long getYieldExpiration() {
-        return yieldExpiration.get();
-    }
-
-    @Override
-    public long getPenalizationPeriod(final TimeUnit timeUnit) {
-        return FormatUtils.getTimeDuration(getPenalizationPeriod(), timeUnit == null ? DEFAULT_TIME_UNIT : timeUnit);
-    }
-
-    @Override
-    public String getPenalizationPeriod() {
-        return penalizationPeriod.get();
-    }
-
-    @Override
-    public void setPenalizationPeriod(final String penalizationPeriod) {
-        writeLock.lock();
-        try {
-            if (isRunning()) {
-                throw new IllegalStateException("Cannot modify Processor configuration while the Processor is running");
-            }
-            final long penalizationMillis = FormatUtils.getTimeDuration(requireNonNull(penalizationPeriod), TimeUnit.MILLISECONDS);
-            if (penalizationMillis < 0) {
-                throw new IllegalArgumentException("Penalization duration must be positive");
-            }
-            this.penalizationPeriod.set(penalizationPeriod);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * Determines the number of concurrent tasks that may be running for this
-     * processor.
-     *
-     * @param taskCount a number of concurrent tasks this processor may have
-     * running
-     * @throws IllegalArgumentException if the given value is less than 1
-     */
-    @Override
-    public void setMaxConcurrentTasks(final int taskCount) {
-        writeLock.lock();
-        try {
-            if (isRunning()) {
-                throw new IllegalStateException("Cannot modify Processor configuration while the Processor is running");
-            }
-            if (taskCount < 1 && getSchedulingStrategy() != SchedulingStrategy.EVENT_DRIVEN) {
-                throw new IllegalArgumentException();
-            }
-            if (!triggeredSerially) {
-                concurrentTaskCount.set(taskCount);
-            }
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    public boolean isTriggeredSerially() {
-        return triggeredSerially;
-    }
-
-    /**
-     * @return the number of tasks that may execute concurrently for this
-     * processor
-     */
-    @Override
-    public int getMaxConcurrentTasks() {
-        return concurrentTaskCount.get();
-    }
-
-    public LogLevel getBulletinLevel() {
-        return LogRepositoryFactory.getRepository(getIdentifier()).getObservationLevel(BULLETIN_OBSERVER_ID);
-    }
-
-    public void setBulletinLevel(final LogLevel level) {
-        LogRepositoryFactory.getRepository(getIdentifier()).setObservationLevel(BULLETIN_OBSERVER_ID, level);
-    }
-
-    @Override
-    public Set<Connection> getConnections() {
-        final Set<Connection> allConnections = new HashSet<>();
-        readLock.lock();
-        try {
-            for (final Set<Connection> connectionSet : connections.values()) {
-                allConnections.addAll(connectionSet);
-            }
-        } finally {
-            readLock.unlock();
-        }
-
-        return allConnections;
-    }
-
-    @Override
-    public List<Connection> getIncomingConnections() {
-        return incomingConnectionsRef.get();
-    }
-
-    @Override
-    public Set<Connection> getConnections(final Relationship relationship) {
-        final Set<Connection> applicableConnections;
-        readLock.lock();
-        try {
-            applicableConnections = connections.get(relationship);
-        } finally {
-            readLock.unlock();
-        }
-        return (applicableConnections == null) ? Collections.<Connection>emptySet() : Collections.unmodifiableSet(applicableConnections);
-    }
-
-    @Override
-    public void addConnection(final Connection connection) {
-        Objects.requireNonNull(connection, "connection cannot be null");
-
-        if (!connection.getSource().equals(this) && !connection.getDestination().equals(this)) {
-            throw new IllegalStateException("Cannot a connection to a ProcessorNode for which the ProcessorNode is neither the Source nor the Destination");
-        }
-
-        writeLock.lock();
-        try {
-            List<Connection> updatedIncoming = null;
-            if (connection.getDestination().equals(this)) {
-                // don't add the connection twice. This may occur if we have a self-loop because we will be told
-                // to add the connection once because we are the source and again because we are the destination.
-                final List<Connection> incomingConnections = incomingConnectionsRef.get();
-                updatedIncoming = new ArrayList<>(incomingConnections);
-                if (!updatedIncoming.contains(connection)) {
-                    updatedIncoming.add(connection);
-                }
-            }
-
-            if (connection.getSource().equals(this)) {
-                // don't add the connection twice. This may occur if we have a self-loop because we will be told
-                // to add the connection once because we are the source and again because we are the destination.
-                if (!destinations.containsKey(connection)) {
-                    for (final Relationship relationship : connection.getRelationships()) {
-                        final Relationship rel = getRelationship(relationship.getName());
-                        Set<Connection> set = connections.get(rel);
-                        if (set == null) {
-                            set = new HashSet<>();
-                            connections.put(rel, set);
-                        }
-
-                        set.add(connection);
-
-                        destinations.put(connection, connection.getDestination());
-                    }
-
-                    final Set<Relationship> autoTerminated = this.undefinedRelationshipsToTerminate.get();
-                    if (autoTerminated != null) {
-                        autoTerminated.removeAll(connection.getRelationships());
-                        this.undefinedRelationshipsToTerminate.set(autoTerminated);
-                    }
-                }
-            }
-
-            if (updatedIncoming != null) {
-                incomingConnectionsRef.set(Collections.unmodifiableList(updatedIncoming));
-            }
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public boolean hasIncomingConnection() {
-        return !incomingConnectionsRef.get().isEmpty();
-    }
-
-    @Override
-    public void updateConnection(final Connection connection) throws IllegalStateException {
-        if (requireNonNull(connection).getSource().equals(this)) {
-            writeLock.lock();
-            try {
-                //
-                // update any relationships
-                //
-                // first check if any relations were removed.
-                final List<Relationship> existingRelationships = new ArrayList<>();
-                for (final Map.Entry<Relationship, Set<Connection>> entry : connections.entrySet()) {
-                    if (entry.getValue().contains(connection)) {
-                        existingRelationships.add(entry.getKey());
-                    }
-                }
-
-                for (final Relationship rel : connection.getRelationships()) {
-                    if (!existingRelationships.contains(rel)) {
-                        // relationship was removed. Check if this is legal.
-                        final Set<Connection> connectionsForRelationship = getConnections(rel);
-                        if (connectionsForRelationship != null && connectionsForRelationship.size() == 1 && this.isRunning() && !isAutoTerminated(rel) && getRelationships().contains(rel)) {
-                            // if we are running and we do not terminate undefined relationships and this is the only
-                            // connection that defines the given relationship, and that relationship is required,
-                            // then it is not legal to remove this relationship from this connection.
-                            throw new IllegalStateException("Cannot remove relationship " + rel.getName() + " from Connection because doing so would invalidate Processor " + this + ", which is currently running");
-                        }
-                    }
-                }
-
-                // remove the connection from any list that currently contains
-                for (final Set<Connection> list : connections.values()) {
-                    list.remove(connection);
-                }
-
-                // add the connection in for all relationships listed.
-                for (final Relationship rel : connection.getRelationships()) {
-                    Set<Connection> set = connections.get(rel);
-                    if (set == null) {
-                        set = new HashSet<>();
-                        connections.put(rel, set);
-                    }
-                    set.add(connection);
-                }
-
-                // update to the new destination
-                destinations.put(connection, connection.getDestination());
-
-                final Set<Relationship> autoTerminated = this.undefinedRelationshipsToTerminate.get();
-                if (autoTerminated != null) {
-                    autoTerminated.removeAll(connection.getRelationships());
-                    this.undefinedRelationshipsToTerminate.set(autoTerminated);
-                }
-            } finally {
-                writeLock.unlock();
-            }
-        }
-
-        if (connection.getDestination().equals(this)) {
-            writeLock.lock();
-            try {
-                // update our incoming connections -- we can just remove & re-add the connection to
-                // update the list.
-                final List<Connection> incomingConnections = incomingConnectionsRef.get();
-                final List<Connection> updatedIncoming = new ArrayList<>(incomingConnections);
-                updatedIncoming.remove(connection);
-                updatedIncoming.add(connection);
-                incomingConnectionsRef.set(Collections.unmodifiableList(updatedIncoming));
-            } finally {
-                writeLock.unlock();
-            }
-        }
-    }
-
-    @Override
-    public void removeConnection(final Connection connection) {
-        boolean connectionRemoved = false;
-
-        if (requireNonNull(connection).getSource().equals(this)) {
-            for (final Relationship relationship : connection.getRelationships()) {
-                final Set<Connection> connectionsForRelationship = getConnections(relationship);
-                if ((connectionsForRelationship == null || connectionsForRelationship.size() <= 1) && isRunning()) {
-                    throw new IllegalStateException("This connection cannot be removed because its source is running and removing it will invalidate this processor");
-                }
-            }
-
-            writeLock.lock();
-            try {
-                for (final Set<Connection> connectionList : this.connections.values()) {
-                    connectionList.remove(connection);
-                }
-
-                connectionRemoved = (destinations.remove(connection) != null);
-            } finally {
-                writeLock.unlock();
-            }
-        }
-
-        if (connection.getDestination().equals(this)) {
-            writeLock.lock();
-            try {
-                final List<Connection> incomingConnections = incomingConnectionsRef.get();
-                if (incomingConnections.contains(connection)) {
-                    final List<Connection> updatedIncoming = new ArrayList<>(incomingConnections);
-                    updatedIncoming.remove(connection);
-                    incomingConnectionsRef.set(Collections.unmodifiableList(updatedIncoming));
-                    return;
-                }
-            } finally {
-                writeLock.unlock();
-            }
-        }
-
-        if (!connectionRemoved) {
-            throw new IllegalArgumentException("Cannot remove a connection from a ProcessorNode for which the ProcessorNode is not the Source");
-        }
-    }
-
-    /**
-     * Gets the relationship for this nodes processor for the given name or
-     * creates a new relationship for the given name.
-     *
-     * @param relationshipName
-     * @return
-     */
-    @Override
-    public Relationship getRelationship(final String relationshipName) {
-        final Relationship specRel = new Relationship.Builder().name(relationshipName).build();
-        Relationship returnRel = specRel;
-
-        final Set<Relationship> relationships;
-        try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-            relationships = processor.getRelationships();
-        }
-
-        for (final Relationship rel : relationships) {
-            if (rel.equals(specRel)) {
-                returnRel = rel;
-                break;
-            }
-        }
-        return returnRel;
-    }
-
-    public Processor getProcessor() {
-        return this.processor;
-    }
-
-    /**
-     * Obtains the Set of destination processors for all relationships excluding
-     * any destinations that are this processor itself (self-loops)
-     *
-     * @return
-     */
-    public Set<Connectable> getDestinations() {
-        final Set<Connectable> nonSelfDestinations = new HashSet<>();
-        readLock.lock();
-        try {
-            for (final Connectable connectable : destinations.values()) {
-                if (connectable != this) {
-                    nonSelfDestinations.add(connectable);
-                }
-            }
-        } finally {
-            readLock.unlock();
-        }
-        return nonSelfDestinations;
-    }
-
-    public Set<Connectable> getDestinations(final Relationship relationship) {
-        readLock.lock();
-        try {
-            final Set<Connectable> destinationSet = new HashSet<>();
-            final Set<Connection> relationshipConnections = connections.get(relationship);
-            if (relationshipConnections != null) {
-                for (final Connection connection : relationshipConnections) {
-                    destinationSet.add(destinations.get(connection));
-                }
-            }
-            return destinationSet;
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    public Set<Relationship> getUndefinedRelationships() {
-        final Set<Relationship> undefined = new HashSet<>();
-        readLock.lock();
-        try {
-            final Set<Relationship> relationships;
-            try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                relationships = processor.getRelationships();
-            }
-
-            if (relationships == null) {
-                return undefined;
-            }
-            for (final Relationship relation : relationships) {
-                final Set<Connection> connectionSet = this.connections.get(relation);
-                if (connectionSet == null || connectionSet.isEmpty()) {
-                    undefined.add(relation);
-                }
-            }
-        } finally {
-            readLock.unlock();
-        }
-        return undefined;
-    }
-
-    /**
-     * Determines if the given node is a destination for this node
-     *
-     * @param node
-     * @return true if is a direct destination node; false otherwise
-     */
-    boolean isRelated(final ProcessorNode node) {
-        readLock.lock();
-        try {
-            return this.destinations.containsValue(node);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public boolean isRunning() {
-        readLock.lock();
-        try {
-            return getScheduledState().equals(ScheduledState.RUNNING) || processScheduler.getActiveThreadCount(this) > 0;
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public boolean isValid() {
-        readLock.lock();
-        try {
-            final ValidationContext validationContext = validationContextFactory.newValidationContext(getProperties(), getAnnotationData());
-
-            final Collection<ValidationResult> validationResults;
-            try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                validationResults = getProcessor().validate(validationContext);
-            }
-
-            for (final ValidationResult result : validationResults) {
-                if (!result.isValid()) {
-                    return false;
-                }
-            }
-
-            for (final Relationship undef : getUndefinedRelationships()) {
-                if (!isAutoTerminated(undef)) {
-                    return false;
-                }
-            }
-        } catch (final Throwable t) {
-            return false;
-        } finally {
-            readLock.unlock();
-        }
-
-        return true;
-    }
-
-    @Override
-    public Collection<ValidationResult> getValidationErrors() {
-        final List<ValidationResult> results = new ArrayList<>();
-        readLock.lock();
-        try {
-            final ValidationContext validationContext = validationContextFactory.newValidationContext(getProperties(), getAnnotationData());
-
-            final Collection<ValidationResult> validationResults;
-            try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                validationResults = getProcessor().validate(validationContext);
-            }
-
-            for (final ValidationResult result : validationResults) {
-                if (!result.isValid()) {
-                    results.add(result);
-                }
-            }
-
-            for (final Relationship relationship : getUndefinedRelationships()) {
-                if (!isAutoTerminated(relationship)) {
-                    final ValidationResult error = new ValidationResult.Builder()
-                            .explanation("Relationship '" + relationship.getName() + "' is not connected to any component and is not auto-terminated")
-                            .subject("Relationship " + relationship.getName())
-                            .valid(false)
-                            .build();
-                    results.add(error);
-                }
-            }
-        } catch (final Throwable t) {
-            results.add(new ValidationResult.Builder().explanation("Failed to run validation due to " + t.toString()).valid(false).build());
-        } finally {
-            readLock.unlock();
-        }
-        return results;
-    }
-
-    /**
-     * Establishes node equality (based on the processor's identifier)
-     *
-     * @param other
-     * @return
-     */
-    @Override
-    public boolean equals(final Object other) {
-        if (!(other instanceof ProcessorNode)) {
-            return false;
-        }
-        final ProcessorNode on = (ProcessorNode) other;
-        return new EqualsBuilder().append(identifier.get(), on.getIdentifier()).isEquals();
-    }
-
-    @Override
-    public int hashCode() {
-        return new HashCodeBuilder(7, 67).append(identifier).toHashCode();
-    }
-
-    @Override
-    public Collection<Relationship> getRelationships() {
-        try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-            return getProcessor().getRelationships();
-        }
-    }
-
-    @Override
-    public String toString() {
-        try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-            return getProcessor().toString();
-        }
-    }
-
-    @Override
-    public ProcessGroup getProcessGroup() {
-        return processGroup.get();
-    }
-
-    @Override
-    public void setProcessGroup(final ProcessGroup group) {
-        writeLock.lock();
-        try {
-            this.processGroup.set(group);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
-        try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-            processor.onTrigger(context, sessionFactory);
-        }
-    }
-
-    @Override
-    public ConnectableType getConnectableType() {
-        return ConnectableType.PROCESSOR;
-    }
-
-    public void setScheduledState(final ScheduledState scheduledState) {
-        this.scheduledState.set(scheduledState);
-        if (!scheduledState.equals(ScheduledState.RUNNING)) {   // if user stops processor, clear yield expiration
-            yieldExpiration.set(0L);
-        }
-    }
-
-    @Override
-    public void setAnnotationData(final String data) {
-        writeLock.lock();
-        try {
-            if (isRunning()) {
-                throw new IllegalStateException("Cannot set AnnotationData while processor is running");
-            }
-
-            this.annotationData.set(data);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public String getAnnotationData() {
-        return annotationData.get();
-    }
-
-    @Override
-    public Collection<ValidationResult> validate(final ValidationContext validationContext) {
-        return processor.validate(validationContext);
-    }
-
-    @Override
-    public void verifyCanDelete() throws IllegalStateException {
-        verifyCanDelete(false);
-    }
-
-    @Override
-    public void verifyCanDelete(final boolean ignoreConnections) {
-        readLock.lock();
-        try {
-            if (isRunning()) {
-                throw new IllegalStateException(this + " is running");
-            }
-
-            if (!ignoreConnections) {
-                for (final Set<Connection> connectionSet : connections.values()) {
-                    for (final Connection connection : connectionSet) {
-                        connection.verifyCanDelete();
-                    }
-                }
-
-                for (final Connection connection : incomingConnectionsRef.get()) {
-                    if (connection.getSource().equals(this)) {
-                        connection.verifyCanDelete();
-                    } else {
-                        throw new IllegalStateException(this + " is the destination of another component");
-                    }
-                }
-            }
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public void verifyCanStart() {
-        readLock.lock();
-        try {
-            if (scheduledState.get() != ScheduledState.STOPPED) {
-                throw new IllegalStateException(this + " is not stopped");
-            }
-            verifyNoActiveThreads();
-
-            if (!isValid()) {
-                throw new IllegalStateException(this + " is not in a valid state");
-            }
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public void verifyCanStop() {
-        if (getScheduledState() != ScheduledState.RUNNING) {
-            throw new IllegalStateException(this + " is not scheduled to run");
-        }
-    }
-
-    @Override
-    public void verifyCanUpdate() {
-        readLock.lock();
-        try {
-            if (isRunning()) {
-                throw new IllegalStateException(this + " is not stopped");
-            }
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public void verifyCanEnable() {
-        readLock.lock();
-        try {
-            if (getScheduledState() != ScheduledState.DISABLED) {
-                throw new IllegalStateException(this + " is not disabled");
-            }
-
-            verifyNoActiveThreads();
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public void verifyCanDisable() {
-        readLock.lock();
-        try {
-            if (getScheduledState() != ScheduledState.STOPPED) {
-                throw new IllegalStateException(this + " is not stopped");
-            }
-            verifyNoActiveThreads();
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    private void verifyNoActiveThreads() throws IllegalStateException {
-        final int threadCount = processScheduler.getActiveThreadCount(this);
-        if (threadCount > 0) {
-            throw new IllegalStateException(this + " has " + threadCount + " threads still active");
-        }
-    }
-
-    @Override
-    public void verifyModifiable() throws IllegalStateException {
-        if (isRunning()) {
-            throw new IllegalStateException("Cannot modify Processor configuration while the Processor is running");
-        }
-    }
-}


[25/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.java
deleted file mode 100644
index 61c3c33..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.java
+++ /dev/null
@@ -1,141 +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.api.dto;
-
-import java.util.LinkedHashSet;
-import java.util.Set;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * The contents of a flow snippet.
- */
-@XmlType(name = "flowSnippet")
-public class FlowSnippetDTO {
-
-    private Set<ProcessGroupDTO> processGroups = new LinkedHashSet<>();
-    private Set<RemoteProcessGroupDTO> remoteProcessGroups = new LinkedHashSet<>();
-    private Set<ProcessorDTO> processors = new LinkedHashSet<>();
-    private Set<PortDTO> inputPorts = new LinkedHashSet<>();
-    private Set<PortDTO> outputPorts = new LinkedHashSet<>();
-    private Set<ConnectionDTO> connections = new LinkedHashSet<>();
-    private Set<LabelDTO> labels = new LinkedHashSet<>();
-    private Set<FunnelDTO> funnels = new LinkedHashSet<>();
-
-    /**
-     * The connections in this flow snippet.
-     *
-     * @return
-     */
-    public Set<ConnectionDTO> getConnections() {
-        return connections;
-    }
-
-    public void setConnections(Set<ConnectionDTO> connections) {
-        this.connections = connections;
-    }
-
-    /**
-     * The input ports in this flow snippet.
-     *
-     * @return
-     */
-    public Set<PortDTO> getInputPorts() {
-        return inputPorts;
-    }
-
-    public void setInputPorts(Set<PortDTO> inputPorts) {
-        this.inputPorts = inputPorts;
-    }
-
-    /**
-     * The labels in this flow snippet.
-     *
-     * @return
-     */
-    public Set<LabelDTO> getLabels() {
-        return labels;
-    }
-
-    public void setLabels(Set<LabelDTO> labels) {
-        this.labels = labels;
-    }
-
-    /**
-     * The funnels in this flow snippet.
-     *
-     * @return
-     */
-    public Set<FunnelDTO> getFunnels() {
-        return funnels;
-    }
-
-    public void setFunnels(Set<FunnelDTO> funnels) {
-        this.funnels = funnels;
-    }
-
-    /**
-     * The output ports in this flow snippet.
-     *
-     * @return
-     */
-    public Set<PortDTO> getOutputPorts() {
-        return outputPorts;
-    }
-
-    public void setOutputPorts(Set<PortDTO> outputPorts) {
-        this.outputPorts = outputPorts;
-    }
-
-    /**
-     * The process groups in this flow snippet.
-     *
-     * @return
-     */
-    public Set<ProcessGroupDTO> getProcessGroups() {
-        return processGroups;
-    }
-
-    public void setProcessGroups(Set<ProcessGroupDTO> processGroups) {
-        this.processGroups = processGroups;
-    }
-
-    /**
-     * The processors in this flow group.
-     *
-     * @return
-     */
-    public Set<ProcessorDTO> getProcessors() {
-        return processors;
-    }
-
-    public void setProcessors(Set<ProcessorDTO> processors) {
-        this.processors = processors;
-    }
-
-    /**
-     * The remote process groups in this flow snippet.
-     *
-     * @return
-     */
-    public Set<RemoteProcessGroupDTO> getRemoteProcessGroups() {
-        return remoteProcessGroups;
-    }
-
-    public void setRemoteProcessGroups(Set<RemoteProcessGroupDTO> remoteProcessGroups) {
-        this.remoteProcessGroups = remoteProcessGroups;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/FunnelDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/FunnelDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/FunnelDTO.java
deleted file mode 100644
index 1240501..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/FunnelDTO.java
+++ /dev/null
@@ -1,29 +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.api.dto;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Details of a funnel.
- */
-@XmlType(name = "funnel")
-public class FunnelDTO extends NiFiComponentDTO {
-
-    public FunnelDTO() {
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/LabelDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/LabelDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/LabelDTO.java
deleted file mode 100644
index f50c792..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/LabelDTO.java
+++ /dev/null
@@ -1,94 +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.api.dto;
-
-import java.util.Collections;
-import java.util.Map;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Details of a label.
- */
-@XmlType(name = "label")
-public class LabelDTO extends NiFiComponentDTO {
-
-    private String label;
-
-    private Double width;
-    private Double height;
-
-    // font-size = 12px
-    // color = #eee
-    private Map<String, String> style = Collections.emptyMap();
-
-    public LabelDTO() {
-    }
-
-    /**
-     * The text that appears in the label.
-     *
-     * @return The label text
-     */
-    public String getLabel() {
-        return label;
-    }
-
-    public void setLabel(final String label) {
-        this.label = label;
-    }
-
-    /**
-     * The style for this label.
-     *
-     * @return
-     */
-    public Map<String, String> getStyle() {
-        return style;
-    }
-
-    public void setStyle(Map<String, String> style) {
-        this.style = style;
-    }
-
-    /**
-     * The height of the label in pixels when at a 1:1 scale.
-     *
-     * @return
-     */
-    public Double getHeight() {
-        return height;
-    }
-
-    public void setHeight(Double height) {
-        this.height = height;
-    }
-
-    /**
-     * The width of the label in pixels when at a 1:1 scale.
-     *
-     * @return
-     */
-    public Double getWidth() {
-        return width;
-    }
-
-    public void setWidth(Double width) {
-        this.width = width;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.java
deleted file mode 100644
index e3c8445..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.java
+++ /dev/null
@@ -1,95 +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.api.dto;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Base class for all nifi components.
- */
-@XmlType(name = "nifiComponent")
-public class NiFiComponentDTO {
-
-    private String id;
-    private String uri;
-    private PositionDTO position;
-    private String parentGroupId;
-
-    public NiFiComponentDTO() {
-    }
-
-    public NiFiComponentDTO(final String id) {
-        this.id = id;
-    }
-
-    public NiFiComponentDTO(final String id, final double x, final double y) {
-        this.id = id;
-        this.position = new PositionDTO(x, y);
-    }
-
-    /**
-     * The id for this component.
-     *
-     * @return The id
-     */
-    public String getId() {
-        return this.id;
-    }
-
-    public void setId(final String id) {
-        this.id = id;
-    }
-
-    /**
-     * The id for the parent group of this component.
-     *
-     * @return
-     */
-    public String getParentGroupId() {
-        return parentGroupId;
-    }
-
-    public void setParentGroupId(String parentGroupId) {
-        this.parentGroupId = parentGroupId;
-    }
-
-    /**
-     * The uri for linking to this component in this NiFi.
-     *
-     * @return The uri
-     */
-    public String getUri() {
-        return uri;
-    }
-
-    public void setUri(String uri) {
-        this.uri = uri;
-    }
-
-    public void setPosition(final PositionDTO position) {
-        this.position = position;
-    }
-
-    /**
-     * The position of this component in the UI.
-     *
-     * @return The position
-     */
-    public PositionDTO getPosition() {
-        return position;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeDTO.java
deleted file mode 100644
index 9499c2e..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeDTO.java
+++ /dev/null
@@ -1,188 +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.api.dto;
-
-import java.util.Date;
-import java.util.List;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.DateTimeAdapter;
-
-/**
- * Details of a node within this NiFi.
- */
-@XmlType(name = "node")
-public class NodeDTO {
-
-    private String nodeId;
-    private String address;
-    private Integer apiPort;
-    private String status;
-    private Date heartbeat;
-    private Date connectionRequested;
-    private Boolean primary;
-    private Integer activeThreadCount;
-    private String queued;
-    private List<NodeEventDTO> events;
-    private Date nodeStartTime;
-
-    /**
-     * The node's last heartbeat timestamp.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(DateTimeAdapter.class)
-    public Date getHeartbeat() {
-        return heartbeat;
-    }
-
-    public void setHeartbeat(Date heartbeat) {
-        this.heartbeat = heartbeat;
-    }
-
-    /**
-     * The time of the node's last connection request.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(DateTimeAdapter.class)
-    public Date getConnectionRequested() {
-        return connectionRequested;
-    }
-
-    public void setConnectionRequested(Date connectionRequested) {
-        this.connectionRequested = connectionRequested;
-    }
-
-    /**
-     * The active thread count.
-     *
-     * @return The active thread count
-     */
-    public Integer getActiveThreadCount() {
-        return activeThreadCount;
-    }
-
-    public void setActiveThreadCount(Integer activeThreadCount) {
-        this.activeThreadCount = activeThreadCount;
-    }
-
-    /**
-     * The queue for the controller.
-     *
-     * @return
-     */
-    public String getQueued() {
-        return queued;
-    }
-
-    public void setQueued(String queued) {
-        this.queued = queued;
-    }
-
-    /**
-     * The node's host/IP address.
-     *
-     * @return
-     */
-    public String getAddress() {
-        return address;
-    }
-
-    public void setAddress(String address) {
-        this.address = address;
-    }
-
-    /**
-     * The node ID.
-     *
-     * @return
-     */
-    public String getNodeId() {
-        return nodeId;
-    }
-
-    public void setNodeId(String nodeId) {
-        this.nodeId = nodeId;
-    }
-
-    /**
-     * The port the node is listening for API requests.
-     *
-     * @return
-     */
-    public Integer getApiPort() {
-        return apiPort;
-    }
-
-    public void setApiPort(Integer port) {
-        this.apiPort = port;
-    }
-
-    /**
-     * The node's status.
-     *
-     * @return
-     */
-    public String getStatus() {
-        return status;
-    }
-
-    public void setStatus(String status) {
-        this.status = status;
-    }
-
-    /**
-     * The node's events.
-     *
-     * @return
-     */
-    public List<NodeEventDTO> getEvents() {
-        return events;
-    }
-
-    public void setEvents(List<NodeEventDTO> events) {
-        this.events = events;
-    }
-
-    /**
-     * Whether this node is the primary node within the cluster.
-     *
-     * @return
-     */
-    public Boolean isPrimary() {
-        return primary;
-    }
-
-    public void setPrimary(Boolean primary) {
-        this.primary = primary;
-    }
-
-    /**
-     * The time at which this Node was last restarted
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(DateTimeAdapter.class)
-    public Date getNodeStartTime() {
-        return nodeStartTime;
-    }
-
-    public void setNodeStartTime(Date nodeStartTime) {
-        this.nodeStartTime = nodeStartTime;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeEventDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeEventDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeEventDTO.java
deleted file mode 100644
index 3cad8d8..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeEventDTO.java
+++ /dev/null
@@ -1,74 +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.api.dto;
-
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.DateTimeAdapter;
-
-/**
- * A event for a node within this NiFi cluster.
- */
-@XmlType(name = "nodeEvent")
-public class NodeEventDTO {
-
-    private Date timestamp;
-    private String category;
-    private String message;
-
-    /**
-     * The category of the node event.
-     *
-     * @return
-     */
-    public String getCategory() {
-        return category;
-    }
-
-    public void setCategory(String category) {
-        this.category = category;
-    }
-
-    /**
-     * The message of the node event.
-     *
-     * @return
-     */
-    public String getMessage() {
-        return message;
-    }
-
-    public void setMessage(String message) {
-        this.message = message;
-    }
-
-    /**
-     * The timestamp of the node event.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(DateTimeAdapter.class)
-    public Date getTimestamp() {
-        return timestamp;
-    }
-
-    public void setTimestamp(Date timestamp) {
-        this.timestamp = timestamp;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsDTO.java
deleted file mode 100644
index 8c83331..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsDTO.java
+++ /dev/null
@@ -1,56 +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.api.dto;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * The system diagnostics for a node with this NiFi cluster.
- */
-@XmlType(name = "nodeSystemDiagnostics")
-public class NodeSystemDiagnosticsDTO {
-
-    private NodeDTO node;
-    private SystemDiagnosticsDTO systemDiagnostics;
-
-    /**
-     * The node.
-     *
-     * @return
-     */
-    public NodeDTO getNode() {
-        return node;
-    }
-
-    public void setNode(NodeDTO node) {
-        this.node = node;
-    }
-
-    /**
-     * The system diagnostics.
-     *
-     * @return
-     */
-    public SystemDiagnosticsDTO getSystemDiagnostics() {
-        return systemDiagnostics;
-    }
-
-    public void setControllerStatus(SystemDiagnosticsDTO systemDiagnostics) {
-        this.systemDiagnostics = systemDiagnostics;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PortDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PortDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PortDTO.java
deleted file mode 100644
index 2a372f4..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PortDTO.java
+++ /dev/null
@@ -1,161 +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.api.dto;
-
-import java.util.Collection;
-import java.util.Set;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * The details for a port within this NiFi flow.
- */
-@XmlType(name = "port")
-public class PortDTO extends NiFiComponentDTO {
-
-    private String name;
-    private String comments;
-    private String state;
-    private String type;
-    private Boolean transmitting;
-    private Integer concurrentlySchedulableTaskCount;
-    private Set<String> userAccessControl;
-    private Set<String> groupAccessControl;
-
-    private Collection<String> validationErrors;
-
-    /**
-     * The name of this port.
-     *
-     * @return
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(final String name) {
-        this.name = name;
-    }
-
-    /**
-     * The state of this port. Possible states are 'RUNNING', 'STOPPED', and
-     * 'DISABLED'.
-     *
-     * @return
-     */
-    public String getState() {
-        return state;
-    }
-
-    public void setState(String state) {
-        this.state = state;
-    }
-
-    /**
-     * The type of port. Possible values are 'INPUT_PORT' or 'OUTPUT_PORT'.
-     *
-     * @return
-     */
-    public String getType() {
-        return type;
-    }
-
-    public void setType(String type) {
-        this.type = type;
-    }
-
-    /**
-     * The number of tasks that should be concurrently scheduled for this port.
-     *
-     * @return
-     */
-    public Integer getConcurrentlySchedulableTaskCount() {
-        return concurrentlySchedulableTaskCount;
-    }
-
-    public void setConcurrentlySchedulableTaskCount(Integer concurrentlySchedulableTaskCount) {
-        this.concurrentlySchedulableTaskCount = concurrentlySchedulableTaskCount;
-    }
-
-    /**
-     * The comments for this port.
-     *
-     * @return
-     */
-    public String getComments() {
-        return comments;
-    }
-
-    public void setComments(String comments) {
-        this.comments = comments;
-    }
-
-    /**
-     * Whether this port has incoming or outgoing connections to a remote NiFi.
-     * This is only applicable when the port is running on the root group.
-     *
-     * @return
-     */
-    public Boolean isTransmitting() {
-        return transmitting;
-    }
-
-    public void setTransmitting(Boolean transmitting) {
-        this.transmitting = transmitting;
-    }
-
-    /**
-     * Groups that are allowed to access this port.
-     *
-     * @return
-     */
-    public Set<String> getGroupAccessControl() {
-        return groupAccessControl;
-    }
-
-    public void setGroupAccessControl(Set<String> groupAccessControl) {
-        this.groupAccessControl = groupAccessControl;
-    }
-
-    /**
-     * Users that are allowed to access this port.
-     *
-     * @return
-     */
-    public Set<String> getUserAccessControl() {
-        return userAccessControl;
-    }
-
-    public void setUserAccessControl(Set<String> userAccessControl) {
-        this.userAccessControl = userAccessControl;
-    }
-
-    /**
-     * Gets the validation errors from this port. These validation errors
-     * represent the problems with the port that must be resolved before it can
-     * be started.
-     *
-     * @return The validation errors
-     */
-    public Collection<String> getValidationErrors() {
-        return validationErrors;
-    }
-
-    public void setValidationErrors(Collection<String> validationErrors) {
-        this.validationErrors = validationErrors;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PositionDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PositionDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PositionDTO.java
deleted file mode 100644
index ab077f3..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PositionDTO.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.web.api.dto;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * A position on the canvas.
- */
-@XmlType(name = "position")
-public class PositionDTO {
-
-    private Double x;
-    private Double y;
-
-    public PositionDTO() {
-    }
-
-    public PositionDTO(Double x, Double y) {
-        this.x = x;
-        this.y = y;
-    }
-
-    /* getters / setters */
-    /**
-     * The x coordinate.
-     *
-     * @return
-     */
-    public Double getX() {
-        return x;
-    }
-
-    public void setX(Double x) {
-        this.x = x;
-    }
-
-    /**
-     * The y coordinate.
-     *
-     * @return
-     */
-    public Double getY() {
-        return y;
-    }
-
-    public void setY(Double y) {
-        this.y = y;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PreviousValueDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PreviousValueDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PreviousValueDTO.java
deleted file mode 100644
index fb33c67..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PreviousValueDTO.java
+++ /dev/null
@@ -1,73 +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.api.dto;
-
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.DateTimeAdapter;
-
-/**
- * The previous value for a processor property.
- */
-@XmlType(name = "previousValue")
-public class PreviousValueDTO {
-
-    private String previousValue;
-    private Date timestamp;
-    private String userName;
-
-    /**
-     * The previous value.
-     *
-     * @return
-     */
-    public String getPreviousValue() {
-        return previousValue;
-    }
-
-    public void setPreviousValue(String previousValue) {
-        this.previousValue = previousValue;
-    }
-
-    /**
-     * When it was modified.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(DateTimeAdapter.class)
-    public Date getTimestamp() {
-        return timestamp;
-    }
-
-    public void setTimestamp(Date timestamp) {
-        this.timestamp = timestamp;
-    }
-
-    /**
-     * The user who changed the previous value.
-     *
-     * @return
-     */
-    public String getUserName() {
-        return userName;
-    }
-
-    public void setUserName(String userName) {
-        this.userName = userName;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java
deleted file mode 100644
index 4140046..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java
+++ /dev/null
@@ -1,219 +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.api.dto;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * The details for a process group within this NiFi flow.
- */
-@XmlType(name = "processGroup")
-public class ProcessGroupDTO extends NiFiComponentDTO {
-
-    private String name;
-    private String comments;
-    private Boolean running;
-
-    private ProcessGroupDTO parent;
-
-    private Integer runningCount;
-    private Integer stoppedCount;
-    private Integer invalidCount;
-    private Integer disabledCount;
-    private Integer activeRemotePortCount;
-    private Integer inactiveRemotePortCount;
-
-    private Integer inputPortCount;
-    private Integer outputPortCount;
-
-    private FlowSnippetDTO contents;
-
-    public ProcessGroupDTO() {
-        super();
-    }
-
-    /**
-     * The name of this Process Group.
-     *
-     * @return The name of this Process Group
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * This Process Group's parent
-     *
-     * @return This Process Group's parent
-     */
-    public ProcessGroupDTO getParent() {
-        return parent;
-    }
-
-    public void setParent(ProcessGroupDTO parent) {
-        this.parent = parent;
-    }
-
-    /**
-     * The comments for this process group.
-     *
-     * @return
-     */
-    public String getComments() {
-        return comments;
-    }
-
-    public void setComments(String comments) {
-        this.comments = comments;
-    }
-
-    /**
-     * The contents of this process group. This field will be populated if the
-     * request is marked verbose.
-     *
-     * @return
-     */
-    public FlowSnippetDTO getContents() {
-        return contents;
-    }
-
-    public void setContents(FlowSnippetDTO contents) {
-        this.contents = contents;
-    }
-
-    /**
-     * The number of input ports contained in this process group.
-     *
-     * @return
-     */
-    public Integer getInputPortCount() {
-        return inputPortCount;
-    }
-
-    public void setInputPortCount(Integer inputPortCount) {
-        this.inputPortCount = inputPortCount;
-    }
-
-    /**
-     * The number of invalid components in this process group.
-     *
-     * @return
-     */
-    public Integer getInvalidCount() {
-        return invalidCount;
-    }
-
-    public void setInvalidCount(Integer invalidCount) {
-        this.invalidCount = invalidCount;
-    }
-
-    /**
-     * The number of output ports in this process group.
-     *
-     * @return
-     */
-    public Integer getOutputPortCount() {
-        return outputPortCount;
-    }
-
-    public void setOutputPortCount(Integer outputPortCount) {
-        this.outputPortCount = outputPortCount;
-    }
-
-    /**
-     * Used in requests, indicates whether this process group should be running.
-     *
-     * @return
-     */
-    public Boolean isRunning() {
-        return running;
-    }
-
-    public void setRunning(Boolean running) {
-        this.running = running;
-    }
-
-    /**
-     * The number of running component in this process group.
-     *
-     * @return
-     */
-    public Integer getRunningCount() {
-        return runningCount;
-    }
-
-    public void setRunningCount(Integer runningCount) {
-        this.runningCount = runningCount;
-    }
-
-    /**
-     * The number of stopped components in this process group.
-     *
-     * @return
-     */
-    public Integer getStoppedCount() {
-        return stoppedCount;
-    }
-
-    public void setStoppedCount(Integer stoppedCount) {
-        this.stoppedCount = stoppedCount;
-    }
-
-    /**
-     * The number of disabled components in this process group.
-     *
-     * @return
-     */
-    public Integer getDisabledCount() {
-        return disabledCount;
-    }
-
-    public void setDisabledCount(Integer disabledCount) {
-        this.disabledCount = disabledCount;
-    }
-
-    /**
-     * The number of active remote ports in this process group.
-     *
-     * @return
-     */
-    public Integer getActiveRemotePortCount() {
-        return activeRemotePortCount;
-    }
-
-    public void setActiveRemotePortCount(Integer activeRemotePortCount) {
-        this.activeRemotePortCount = activeRemotePortCount;
-    }
-
-    /**
-     * The number of inactive remote ports in this process group.
-     *
-     * @return
-     */
-    public Integer getInactiveRemotePortCount() {
-        return inactiveRemotePortCount;
-    }
-
-    public void setInactiveRemotePortCount(Integer inactiveRemotePortCount) {
-        this.inactiveRemotePortCount = inactiveRemotePortCount;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java
deleted file mode 100644
index 1481b0f..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java
+++ /dev/null
@@ -1,486 +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.api.dto;
-
-import java.util.Map;
-import java.util.Set;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Configuration details for a processor in this NiFi.
- */
-@XmlType(name = "processorConfig")
-public class ProcessorConfigDTO {
-
-    private Map<String, String> properties;
-    private Map<String, PropertyDescriptorDTO> descriptors;
-
-    // settings
-    private String schedulingPeriod;
-    private String schedulingStrategy;
-    private String penaltyDuration;
-    private String yieldDuration;
-    private String bulletinLevel;
-    private Long runDurationMillis;
-    private Integer concurrentlySchedulableTaskCount;
-    private Set<String> autoTerminatedRelationships;
-    private String comments;
-    private String customUiUrl;
-    private Boolean lossTolerant;
-
-    // annotation data
-    private String annotationData;
-
-    private Map<String, String> defaultConcurrentTasks;
-    private Map<String, String> defaultSchedulingPeriod;
-
-    public ProcessorConfigDTO() {
-
-    }
-
-    /**
-     * The amount of time that should elapse between task executions. This will
-     * not affect currently scheduled tasks.
-     *
-     * @return The scheduling period in seconds
-     */
-    public String getSchedulingPeriod() {
-        return schedulingPeriod;
-    }
-
-    public void setSchedulingPeriod(String setSchedulingPeriod) {
-        this.schedulingPeriod = setSchedulingPeriod;
-    }
-
-    /**
-     * Indicates whether the processor should be scheduled to run in
-     * event-driven mode or timer-driven mode
-     *
-     * @return
-     */
-    public String getSchedulingStrategy() {
-        return schedulingStrategy;
-    }
-
-    public void setSchedulingStrategy(String schedulingStrategy) {
-        this.schedulingStrategy = schedulingStrategy;
-    }
-
-    /**
-     * The amount of time that is used when this processor penalizes a flow
-     * file.
-     *
-     * @return
-     */
-    public String getPenaltyDuration() {
-        return penaltyDuration;
-    }
-
-    public void setPenaltyDuration(String penaltyDuration) {
-        this.penaltyDuration = penaltyDuration;
-    }
-
-    /**
-     * When yielding, this amount of time must elaspe before this processor is
-     * scheduled again.
-     *
-     * @return
-     */
-    public String getYieldDuration() {
-        return yieldDuration;
-    }
-
-    public void setYieldDuration(String yieldDuration) {
-        this.yieldDuration = yieldDuration;
-    }
-
-    /**
-     * The level at this this processor will report bulletins.
-     *
-     * @return
-     */
-    public String getBulletinLevel() {
-        return bulletinLevel;
-    }
-
-    public void setBulletinLevel(String bulletinLevel) {
-        this.bulletinLevel = bulletinLevel;
-    }
-
-    /**
-     * The number of tasks that should be concurrently scheduled for this
-     * processor. If this processor doesn't allow parallel processing then any
-     * positive input will be ignored.
-     *
-     * @return The concurrently schedulable task count
-     */
-    public Integer getConcurrentlySchedulableTaskCount() {
-        return concurrentlySchedulableTaskCount;
-    }
-
-    public void setConcurrentlySchedulableTaskCount(Integer concurrentlySchedulableTaskCount) {
-        this.concurrentlySchedulableTaskCount = concurrentlySchedulableTaskCount;
-    }
-
-    /**
-     * Whether or not this Processor is Loss Tolerant
-     *
-     * @return
-     */
-    public Boolean isLossTolerant() {
-        return lossTolerant;
-    }
-
-    public void setLossTolerant(final Boolean lossTolerant) {
-        this.lossTolerant = lossTolerant;
-    }
-
-    /**
-     * The comments for this processor.
-     *
-     * @return The comments
-     */
-    public String getComments() {
-        return comments;
-    }
-
-    public void setComments(String comments) {
-        this.comments = comments;
-    }
-
-    /**
-     * The properties for this processor. Properties whose value is not set will
-     * only contain the property name. These properties are (un)marshalled
-     * differently since we need/want to control the ordering of the properties.
-     * The descriptors and metadata are used as a lookup when processing these
-     * properties.
-     *
-     * @return The optional properties
-     */
-    public Map<String, String> getProperties() {
-        return properties;
-    }
-
-    public void setProperties(Map<String, String> properties) {
-        this.properties = properties;
-    }
-
-    /**
-     * The descriptors for this processor's properties.
-     *
-     * @return
-     */
-    public Map<String, PropertyDescriptorDTO> getDescriptors() {
-        return descriptors;
-    }
-
-    public void setDescriptors(Map<String, PropertyDescriptorDTO> descriptors) {
-        this.descriptors = descriptors;
-    }
-
-    /**
-     * Annotation data for this processor.
-     *
-     * @return The annotation data
-     */
-    public String getAnnotationData() {
-        return annotationData;
-    }
-
-    public void setAnnotationData(String annotationData) {
-        this.annotationData = annotationData;
-    }
-
-    /**
-     * Whether of not this processor has a custom UI.
-     *
-     * @return
-     */
-    public String getCustomUiUrl() {
-        return customUiUrl;
-    }
-
-    public void setCustomUiUrl(String customUiUrl) {
-        this.customUiUrl = customUiUrl;
-    }
-
-    /**
-     * The names of all processor relationships that cause a flow file to be
-     * terminated if the relationship is not connected to anything
-     *
-     * @return
-     */
-    public Set<String> getAutoTerminatedRelationships() {
-        return autoTerminatedRelationships;
-    }
-
-    public void setAutoTerminatedRelationships(final Set<String> autoTerminatedRelationships) {
-        this.autoTerminatedRelationships = autoTerminatedRelationships;
-    }
-
-    /**
-     * Maps default values for concurrent tasks for each applicable scheduling
-     * strategy.
-     *
-     * @return
-     */
-    public Map<String, String> getDefaultConcurrentTasks() {
-        return defaultConcurrentTasks;
-    }
-
-    public void setDefaultConcurrentTasks(Map<String, String> defaultConcurrentTasks) {
-        this.defaultConcurrentTasks = defaultConcurrentTasks;
-    }
-
-    /**
-     * The run duration in milliseconds.
-     *
-     * @return
-     */
-    public Long getRunDurationMillis() {
-        return runDurationMillis;
-    }
-
-    public void setRunDurationMillis(Long runDurationMillis) {
-        this.runDurationMillis = runDurationMillis;
-    }
-
-    /**
-     * Maps default values for scheduling period for each applicable scheduling
-     * strategy.
-     *
-     * @return
-     */
-    public Map<String, String> getDefaultSchedulingPeriod() {
-        return defaultSchedulingPeriod;
-    }
-
-    public void setDefaultSchedulingPeriod(Map<String, String> defaultSchedulingPeriod) {
-        this.defaultSchedulingPeriod = defaultSchedulingPeriod;
-    }
-
-    /**
-     * The allowable values for a property with a constrained set of options.
-     */
-    @XmlType(name = "allowableValue")
-    public static class AllowableValueDTO {
-
-        private String displayName;
-        private String value;
-        private String description;
-
-        /**
-         * Returns the human-readable value that is allowed for this
-         * PropertyDescriptor
-         *
-         * @return
-         */
-        public String getDisplayName() {
-            return displayName;
-        }
-
-        public void setDisplayName(String displayName) {
-            this.displayName = displayName;
-        }
-
-        /**
-         * Returns the value for this allowable value.
-         *
-         * @return
-         */
-        public String getValue() {
-            return value;
-        }
-
-        public void setValue(String value) {
-            this.value = value;
-        }
-
-        /**
-         * Returns a description of this Allowable Value, or <code>null</code>
-         * if no description is given
-         *
-         * @return
-         */
-        public String getDescription() {
-            return description;
-        }
-
-        public void setDescription(String description) {
-            this.description = description;
-        }
-
-        @Override
-        public boolean equals(final Object obj) {
-            if (obj == this) {
-                return true;
-            }
-
-            if (!(obj instanceof AllowableValueDTO)) {
-                return false;
-            }
-
-            final AllowableValueDTO other = (AllowableValueDTO) obj;
-            return (this.value.equals(other.getValue()));
-        }
-
-        @Override
-        public int hashCode() {
-            return 23984731 + 17 * value.hashCode();
-        }
-    }
-
-    /**
-     * A description of a processor property.
-     */
-    @XmlType(name = "propertyDescriptor")
-    public static class PropertyDescriptorDTO {
-
-        private String name;
-        private String displayName;
-        private String description;
-        private String defaultValue;
-        private Set<AllowableValueDTO> allowableValues;
-        private boolean required;
-        private boolean sensitive;
-        private boolean dynamic;
-        private boolean supportsEl;
-
-        /**
-         * The set of allowable values for this property. If empty then the
-         * allowable values are not constrained.
-         *
-         * @return
-         */
-        public Set<AllowableValueDTO> getAllowableValues() {
-            return allowableValues;
-        }
-
-        public void setAllowableValues(Set<AllowableValueDTO> allowableValues) {
-            this.allowableValues = allowableValues;
-        }
-
-        /**
-         * The default value for this property.
-         *
-         * @return
-         */
-        public String getDefaultValue() {
-            return defaultValue;
-        }
-
-        public void setDefaultValue(String defaultValue) {
-            this.defaultValue = defaultValue;
-        }
-
-        /**
-         * And explanation of the meaning of the given property. This
-         * description is meant to be displayed to a user or simply provide a
-         * mechanism of documenting intent.
-         *
-         * @return
-         */
-        public String getDescription() {
-            return description;
-        }
-
-        public void setDescription(String description) {
-            this.description = description;
-        }
-
-        /**
-         * The property name.
-         *
-         * @return
-         */
-        public String getName() {
-            return name;
-        }
-
-        public void setName(String name) {
-            this.name = name;
-        }
-
-        /**
-         * The human-readable name to display to users.
-         *
-         * @return
-         */
-        public String getDisplayName() {
-            return displayName;
-        }
-
-        public void setDisplayName(String displayName) {
-            this.displayName = displayName;
-        }
-
-        /**
-         * Determines whether the property is required for this processor.
-         *
-         * @return
-         */
-        public boolean isRequired() {
-            return required;
-        }
-
-        public void setRequired(boolean required) {
-            this.required = required;
-        }
-
-        /**
-         * Indicates that the value for this property should be considered
-         * sensitive and protected whenever stored or represented.
-         *
-         * @return
-         */
-        public boolean isSensitive() {
-            return sensitive;
-        }
-
-        public void setSensitive(boolean sensitive) {
-            this.sensitive = sensitive;
-        }
-
-        /**
-         * Indicates whether this property is dynamic.
-         *
-         * @return
-         */
-        public boolean isDynamic() {
-            return dynamic;
-        }
-
-        public void setDynamic(boolean dynamic) {
-            this.dynamic = dynamic;
-        }
-
-        /**
-         * Specifies whether or not this property support expression language.
-         *
-         * @return
-         */
-        public boolean getSupportsEl() {
-            return supportsEl;
-        }
-
-        public void setSupportsEl(boolean supportsEl) {
-            this.supportsEl = supportsEl;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.java
deleted file mode 100644
index 71ba4ed..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.java
+++ /dev/null
@@ -1,181 +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.api.dto;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Details for a processor within this NiFi.
- */
-@XmlType(name = "processor")
-public class ProcessorDTO extends NiFiComponentDTO {
-
-    private String name;
-    private String type;
-    private String state;
-    private Map<String, String> style;
-    private List<RelationshipDTO> relationships;
-    private String description;
-    private Boolean supportsParallelProcessing;
-    private Boolean supportsEventDriven;
-
-    private ProcessorConfigDTO config;
-
-    private Collection<String> validationErrors;
-
-    public ProcessorDTO() {
-        super();
-    }
-
-    /**
-     * The name of this processor.
-     *
-     * @return This processors name
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * The type of this processor.
-     *
-     * @return This processors type
-     */
-    public String getType() {
-        return type;
-    }
-
-    public void setType(String type) {
-        this.type = type;
-    }
-
-    /**
-     * The state of this processor. Possible states are 'RUNNING', 'STOPPED',
-     * and 'DISABLED'.
-     *
-     * @return
-     */
-    public String getState() {
-        return state;
-    }
-
-    public void setState(String state) {
-        this.state = state;
-    }
-
-    /**
-     * The styles for this processor. (Currently only supports color).
-     *
-     * @return
-     */
-    public Map<String, String> getStyle() {
-        return style;
-    }
-
-    public void setStyle(Map<String, String> style) {
-        this.style = style;
-    }
-
-    /**
-     * Whether this processor supports parallel processing.
-     *
-     * @return
-     */
-    public Boolean getSupportsParallelProcessing() {
-        return supportsParallelProcessing;
-    }
-
-    public void setSupportsParallelProcessing(Boolean supportsParallelProcessing) {
-        this.supportsParallelProcessing = supportsParallelProcessing;
-    }
-
-    /**
-     * Whether this processor supports event driven scheduling.
-     *
-     * @return
-     */
-    public Boolean getSupportsEventDriven() {
-        return supportsEventDriven;
-    }
-
-    public void setSupportsEventDriven(Boolean supportsEventDriven) {
-        this.supportsEventDriven = supportsEventDriven;
-    }
-
-    /**
-     * Gets the available relationships that this processor currently supports.
-     *
-     * @return The available relationships
-     */
-    public List<RelationshipDTO> getRelationships() {
-        return relationships;
-    }
-
-    public void setRelationships(List<RelationshipDTO> relationships) {
-        this.relationships = relationships;
-    }
-
-    /**
-     * The configuration details for this processor. These details will be
-     * included in a response if the verbose flag is set to true.
-     *
-     * @return The processor configuration details
-     */
-    public ProcessorConfigDTO getConfig() {
-        return config;
-    }
-
-    public void setConfig(ProcessorConfigDTO config) {
-        this.config = config;
-    }
-
-    /**
-     * Gets the validation errors from this processor. These validation errors
-     * represent the problems with the processor that must be resolved before it
-     * can be started.
-     *
-     * @return The validation errors
-     */
-    public Collection<String> getValidationErrors() {
-        return validationErrors;
-    }
-
-    public void setValidationErrors(Collection<String> validationErrors) {
-        this.validationErrors = validationErrors;
-    }
-
-    /**
-     * Gets the description for this processor.
-     *
-     * @return
-     */
-    public String getDescription() {
-        return description;
-    }
-
-    public void setDescription(final String description) {
-        this.description = description;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.java
deleted file mode 100644
index 2741116..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.java
+++ /dev/null
@@ -1,56 +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.api.dto;
-
-import java.util.Map;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * History of a processor's properties.
- */
-@XmlType(name = "processorHistory")
-public class ProcessorHistoryDTO {
-
-    private String processorId;
-    private Map<String, PropertyHistoryDTO> propertyHistory;
-
-    /**
-     * The processor id.
-     *
-     * @return
-     */
-    public String getProcessorId() {
-        return processorId;
-    }
-
-    public void setProcessorId(String processorId) {
-        this.processorId = processorId;
-    }
-
-    /**
-     * The history for this processors properties.
-     *
-     * @return
-     */
-    public Map<String, PropertyHistoryDTO> getPropertyHistory() {
-        return propertyHistory;
-    }
-
-    public void setPropertyHistory(Map<String, PropertyHistoryDTO> propertyHistory) {
-        this.propertyHistory = propertyHistory;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyHistoryDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyHistoryDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyHistoryDTO.java
deleted file mode 100644
index 064ad21..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyHistoryDTO.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.web.api.dto;
-
-import java.util.List;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * History of a processor property.
- */
-@XmlType(name = "propertyHistory")
-public class PropertyHistoryDTO {
-
-    private List<PreviousValueDTO> previousValues;
-
-    /**
-     * The previous values.
-     *
-     * @return
-     */
-    public List<PreviousValueDTO> getPreviousValues() {
-        return previousValues;
-    }
-
-    public void setPreviousValues(List<PreviousValueDTO> previousValues) {
-        this.previousValues = previousValues;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RelationshipDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RelationshipDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RelationshipDTO.java
deleted file mode 100644
index 7042aaa..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RelationshipDTO.java
+++ /dev/null
@@ -1,69 +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.api.dto;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Details of a relationship.
- */
-@XmlType(name = "relationship")
-public class RelationshipDTO {
-
-    private String name;
-    private String description;
-    private Boolean autoTerminate;
-
-    /**
-     * The relationship name.
-     *
-     * @return
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * The relationship description.
-     *
-     * @return
-     */
-    public String getDescription() {
-        return description;
-    }
-
-    public void setDescription(String description) {
-        this.description = description;
-    }
-
-    /**
-     * Whether or not this relationship is auto terminated.
-     *
-     * @return
-     */
-    public Boolean isAutoTerminate() {
-        return autoTerminate;
-    }
-
-    public void setAutoTerminate(Boolean autoTerminate) {
-        this.autoTerminate = autoTerminate;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupContentsDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupContentsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupContentsDTO.java
deleted file mode 100644
index 1e5356d..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupContentsDTO.java
+++ /dev/null
@@ -1,56 +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.api.dto;
-
-import java.util.Set;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Contents of a remote process group.
- */
-@XmlType(name = "remoteProcessGroupContents")
-public class RemoteProcessGroupContentsDTO {
-
-    private Set<RemoteProcessGroupPortDTO> inputPorts;
-    private Set<RemoteProcessGroupPortDTO> outputPorts;
-
-    /**
-     * The Controller Input Ports to which data can be sent
-     *
-     * @return
-     */
-    public Set<RemoteProcessGroupPortDTO> getInputPorts() {
-        return inputPorts;
-    }
-
-    public void setInputPorts(Set<RemoteProcessGroupPortDTO> inputPorts) {
-        this.inputPorts = inputPorts;
-    }
-
-    /**
-     * The Controller Output Ports from which data can be retrieved
-     *
-     * @return
-     */
-    public Set<RemoteProcessGroupPortDTO> getOutputPorts() {
-        return outputPorts;
-    }
-
-    public void setOutputPorts(Set<RemoteProcessGroupPortDTO> outputPorts) {
-        this.outputPorts = outputPorts;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupDTO.java
deleted file mode 100644
index df59b13..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupDTO.java
+++ /dev/null
@@ -1,279 +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.api.dto;
-
-import java.util.Date;
-import java.util.List;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.DateTimeAdapter;
-
-/**
- * Details of a remote process group in this NiFi.
- */
-@XmlType(name = "remoteProcessGroup")
-public class RemoteProcessGroupDTO extends NiFiComponentDTO {
-
-    private String targetUri;
-    private Boolean targetSecure;
-
-    private String name;
-    private String comments;
-    private String communicationsTimeout;
-    private String yieldDuration;
-
-    private List<String> authorizationIssues;
-    private Boolean transmitting;
-
-    private Integer inputPortCount;
-    private Integer outputPortCount;
-
-    private Integer activeRemoteInputPortCount;
-    private Integer inactiveRemoteInputPortCount;
-    private Integer activeRemoteOutputPortCount;
-    private Integer inactiveRemoteOutputPortCount;
-
-    private Date flowRefreshed;
-
-    private RemoteProcessGroupContentsDTO contents;
-
-    public RemoteProcessGroupDTO() {
-        super();
-    }
-
-    public RemoteProcessGroupDTO(final RemoteProcessGroupDTO toCopy) {
-        setId(toCopy.getId());
-        setPosition(toCopy.getPosition());
-        targetUri = toCopy.getTargetUri();
-        name = toCopy.getName();
-    }
-
-    public void setTargetUri(final String targetUri) {
-        this.targetUri = targetUri;
-    }
-
-    /**
-     * The target uri of this remote process group.
-     *
-     * @return
-     */
-    public String getTargetUri() {
-        return this.targetUri;
-    }
-
-    /**
-     * The name of this remote process group.
-     *
-     * @param name
-     */
-    public void setName(final String name) {
-        this.name = name;
-    }
-
-    public String getName() {
-        return this.name;
-    }
-
-    /**
-     * Comments for this remote process group.
-     *
-     * @return
-     */
-    public String getComments() {
-        return comments;
-    }
-
-    public void setComments(String comments) {
-        this.comments = comments;
-    }
-
-    /**
-     * Returns any remote authorization issues for this remote process group.
-     *
-     * @return
-     */
-    public List<String> getAuthorizationIssues() {
-        return authorizationIssues;
-    }
-
-    public void setAuthorizationIssues(List<String> authorizationIssues) {
-        this.authorizationIssues = authorizationIssues;
-    }
-
-    /**
-     * Whether or not this remote process group is actively transmitting.
-     *
-     * @return
-     */
-    public Boolean isTransmitting() {
-        return transmitting;
-    }
-
-    public void setTransmitting(Boolean transmitting) {
-        this.transmitting = transmitting;
-    }
-
-    /**
-     * Whether or not the target is running securely.
-     *
-     * @return
-     */
-    public Boolean isTargetSecure() {
-        return targetSecure;
-    }
-
-    public void setTargetSecure(Boolean targetSecure) {
-        this.targetSecure = targetSecure;
-    }
-
-    /**
-     * Returns the time period used for the timeout when communicating with this
-     * RemoteProcessGroup.
-     *
-     * @return
-     */
-    public String getCommunicationsTimeout() {
-        return communicationsTimeout;
-    }
-
-    public void setCommunicationsTimeout(String communicationsTimeout) {
-        this.communicationsTimeout = communicationsTimeout;
-    }
-
-    /**
-     * When yielding, this amount of time must elaspe before this remote process
-     * group is scheduled again.
-     *
-     * @return
-     */
-    public String getYieldDuration() {
-        return yieldDuration;
-    }
-
-    public void setYieldDuration(String yieldDuration) {
-        this.yieldDuration = yieldDuration;
-    }
-
-    /**
-     * The number of active remote input ports.
-     *
-     * @return
-     */
-    public Integer getActiveRemoteInputPortCount() {
-        return activeRemoteInputPortCount;
-    }
-
-    public void setActiveRemoteInputPortCount(Integer activeRemoteInputPortCount) {
-        this.activeRemoteInputPortCount = activeRemoteInputPortCount;
-    }
-
-    /**
-     * The number of inactive remote input ports.
-     *
-     * @return
-     */
-    public Integer getInactiveRemoteInputPortCount() {
-        return inactiveRemoteInputPortCount;
-    }
-
-    public void setInactiveRemoteInputPortCount(Integer inactiveRemoteInputPortCount) {
-        this.inactiveRemoteInputPortCount = inactiveRemoteInputPortCount;
-    }
-
-    /**
-     * The number of active remote output ports.
-     *
-     * @return
-     */
-    public Integer getActiveRemoteOutputPortCount() {
-        return activeRemoteOutputPortCount;
-    }
-
-    public void setActiveRemoteOutputPortCount(Integer activeRemoteOutputPortCount) {
-        this.activeRemoteOutputPortCount = activeRemoteOutputPortCount;
-    }
-
-    /**
-     * The number of inactive remote output ports.
-     *
-     * @return
-     */
-    public Integer getInactiveRemoteOutputPortCount() {
-        return inactiveRemoteOutputPortCount;
-    }
-
-    public void setInactiveRemoteOutputPortCount(Integer inactiveRemoteOutputPortCount) {
-        this.inactiveRemoteOutputPortCount = inactiveRemoteOutputPortCount;
-    }
-
-    /**
-     * The number of Remote Input Ports currently available in the remote NiFi
-     * instance
-     *
-     * @return
-     */
-    public Integer getInputPortCount() {
-        return inputPortCount;
-    }
-
-    public void setInputPortCount(Integer inputPortCount) {
-        this.inputPortCount = inputPortCount;
-    }
-
-    /**
-     * The number of Remote Output Ports currently available in the remote NiFi
-     * instance
-     *
-     * @return
-     */
-    public Integer getOutputPortCount() {
-        return outputPortCount;
-    }
-
-    public void setOutputPortCount(Integer outputPortCount) {
-        this.outputPortCount = outputPortCount;
-    }
-
-    /**
-     * The contents of this remote process group. Will contain available
-     * input/output ports.
-     *
-     * @return
-     */
-    public RemoteProcessGroupContentsDTO getContents() {
-        return contents;
-    }
-
-    public void setContents(RemoteProcessGroupContentsDTO contents) {
-        this.contents = contents;
-    }
-
-    /**
-     * When the flow for this remote group was last refreshed.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(DateTimeAdapter.class)
-    public Date getFlowRefreshed() {
-        return flowRefreshed;
-    }
-
-    public void setFlowRefreshed(Date flowRefreshed) {
-        this.flowRefreshed = flowRefreshed;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupPortDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupPortDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupPortDTO.java
deleted file mode 100644
index 7948dad..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupPortDTO.java
+++ /dev/null
@@ -1,192 +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.api.dto;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Details of a port in a remote process group.
- */
-@XmlType(name = "remoteProcessGroupPort")
-public class RemoteProcessGroupPortDTO {
-
-    private String id;
-    private String groupId;
-    private String name;
-    private String comments;
-    private Integer concurrentlySchedulableTaskCount;
-    private Boolean transmitting;
-    private Boolean useCompression;
-    private Boolean exists;
-    private Boolean targetRunning;
-    private Boolean connected;
-
-    /**
-     * The comments as configured in the target port.
-     *
-     * @return
-     */
-    public String getComments() {
-        return comments;
-    }
-
-    public void setComments(String comments) {
-        this.comments = comments;
-    }
-
-    /**
-     * The number tasks that may transmit flow files to the target port
-     * concurrently.
-     *
-     * @return
-     */
-    public Integer getConcurrentlySchedulableTaskCount() {
-        return concurrentlySchedulableTaskCount;
-    }
-
-    public void setConcurrentlySchedulableTaskCount(Integer concurrentlySchedulableTaskCount) {
-        this.concurrentlySchedulableTaskCount = concurrentlySchedulableTaskCount;
-    }
-
-    /**
-     * The id of the target port.
-     *
-     * @return
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The id of the remote process group that this port resides in.
-     *
-     * @return
-     */
-    public String getGroupId() {
-        return groupId;
-    }
-
-    public void setGroupId(String groupId) {
-        this.groupId = groupId;
-    }
-
-    /**
-     * The name of the target port.
-     *
-     * @return
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * Whether or not this remote group port is configured for transmission.
-     *
-     * @return
-     */
-    public Boolean isTransmitting() {
-        return transmitting;
-    }
-
-    public void setTransmitting(Boolean transmitting) {
-        this.transmitting = transmitting;
-    }
-
-    /**
-     * Whether or not flow file are compressed when sent to this target port.
-     *
-     * @return
-     */
-    public Boolean getUseCompression() {
-        return useCompression;
-    }
-
-    public void setUseCompression(Boolean useCompression) {
-        this.useCompression = useCompression;
-    }
-
-    /**
-     * Whether or not the target port exists.
-     *
-     * @return
-     */
-    public Boolean getExists() {
-        return exists;
-    }
-
-    public void setExists(Boolean exists) {
-        this.exists = exists;
-    }
-
-    /**
-     * Whether or not the target port is running.
-     *
-     * @return
-     */
-    public Boolean isTargetRunning() {
-        return targetRunning;
-    }
-
-    public void setTargetRunning(Boolean targetRunning) {
-        this.targetRunning = targetRunning;
-    }
-
-    /**
-     * Whether or not this port has either an incoming or outgoing connection.
-     *
-     * @return
-     */
-    public Boolean isConnected() {
-        return connected;
-    }
-
-    public void setConnected(Boolean connected) {
-        this.connected = connected;
-    }
-
-    @Override
-    public int hashCode() {
-        return 923847 + String.valueOf(name).hashCode();
-    }
-
-    @Override
-    public boolean equals(final Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (!(obj instanceof RemoteProcessGroupPortDTO)) {
-            return false;
-        }
-        final RemoteProcessGroupPortDTO other = (RemoteProcessGroupPortDTO) obj;
-        if (name == null && other.name == null) {
-            return true;
-        }
-
-        if (name == null) {
-            return false;
-        }
-        return name.equals(other.name);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.java
deleted file mode 100644
index e608a7e..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.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.web.api.dto;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Current revision for this NiFi.
- */
-@XmlType(name = "revision")
-public class RevisionDTO {
-
-    private String clientId;
-    private Long version;
-
-    /* getters / setters */
-    /**
-     * A client identifier used to make a request. By including a client
-     * identifier, the API can allow multiple requests without needing the
-     * current revision. Due to the asynchronous nature of requests/responses
-     * this was implemented to allow the client to make numerous requests
-     * without having to wait for the previous response to come back.
-     *
-     * @return The client id
-     */
-    public String getClientId() {
-        return clientId;
-    }
-
-    public void setClientId(String clientId) {
-        this.clientId = clientId;
-    }
-
-    /**
-     * NiFi employs an optimistic locking strategy where the client must include
-     * a revision in their request when performing an update. In a response,
-     * this field represents the updated base version.
-     *
-     * @return The revision
-     */
-    public Long getVersion() {
-        return version;
-    }
-
-    public void setVersion(Long version) {
-        this.version = version;
-    }
-
-}


[14/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ConnectingNodeMutableRequestException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ConnectingNodeMutableRequestException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ConnectingNodeMutableRequestException.java
deleted file mode 100644
index 365b5f0..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ConnectingNodeMutableRequestException.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.cluster.manager.exception;
-
-/**
- * Represents the exceptional case when a HTTP request that may change a node's
- * dataflow is to be replicated while a node is connecting to the cluster.
- *
- * @author unattributed
- */
-public class ConnectingNodeMutableRequestException extends MutableRequestException {
-
-    public ConnectingNodeMutableRequestException() {
-    }
-
-    public ConnectingNodeMutableRequestException(String msg) {
-        super(msg);
-    }
-
-    public ConnectingNodeMutableRequestException(Throwable cause) {
-        super(cause);
-    }
-
-    public ConnectingNodeMutableRequestException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/DisconnectedNodeMutableRequestException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/DisconnectedNodeMutableRequestException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/DisconnectedNodeMutableRequestException.java
deleted file mode 100644
index 412a555..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/DisconnectedNodeMutableRequestException.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.cluster.manager.exception;
-
-/**
- * Represents the exceptional case when a HTTP request that may change a node's
- * dataflow is to be replicated while one or more nodes are disconnected.
- *
- * @author unattributed
- */
-public class DisconnectedNodeMutableRequestException extends MutableRequestException {
-
-    public DisconnectedNodeMutableRequestException() {
-    }
-
-    public DisconnectedNodeMutableRequestException(String msg) {
-        super(msg);
-    }
-
-    public DisconnectedNodeMutableRequestException(Throwable cause) {
-        super(cause);
-    }
-
-    public DisconnectedNodeMutableRequestException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalClusterStateException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalClusterStateException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalClusterStateException.java
deleted file mode 100644
index 6c4e670..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalClusterStateException.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.cluster.manager.exception;
-
-/**
- * Signals that an operation to be performed on a cluster has been invoked at an
- * illegal or inappropriate time.
- *
- * @author unattributed
- */
-public class IllegalClusterStateException extends ClusterException {
-
-    public IllegalClusterStateException() {
-    }
-
-    public IllegalClusterStateException(String msg) {
-        super(msg);
-    }
-
-    public IllegalClusterStateException(Throwable cause) {
-        super(cause);
-    }
-
-    public IllegalClusterStateException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDeletionException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDeletionException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDeletionException.java
deleted file mode 100644
index adef62a..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDeletionException.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.cluster.manager.exception;
-
-/**
- * Represents the exceptional case when a deletion request is issued to a node
- * that cannot be deleted (e.g., the node is not disconnected).
- *
- * @author unattributed
- */
-public class IllegalNodeDeletionException extends IllegalClusterStateException {
-
-    public IllegalNodeDeletionException() {
-    }
-
-    public IllegalNodeDeletionException(String msg) {
-        super(msg);
-    }
-
-    public IllegalNodeDeletionException(Throwable cause) {
-        super(cause);
-    }
-
-    public IllegalNodeDeletionException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDisconnectionException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDisconnectionException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDisconnectionException.java
deleted file mode 100644
index 7e61b24..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDisconnectionException.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.cluster.manager.exception;
-
-/**
- * Represents the exceptional case when a disconnection request is issued to a
- * node that cannot be disconnected (e.g., last node in cluster, node is primary
- * node).
- *
- * @author unattributed
- */
-public class IllegalNodeDisconnectionException extends IllegalClusterStateException {
-
-    public IllegalNodeDisconnectionException() {
-    }
-
-    public IllegalNodeDisconnectionException(String msg) {
-        super(msg);
-    }
-
-    public IllegalNodeDisconnectionException(Throwable cause) {
-        super(cause);
-    }
-
-    public IllegalNodeDisconnectionException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeReconnectionException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeReconnectionException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeReconnectionException.java
deleted file mode 100644
index 96c76bc..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeReconnectionException.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.cluster.manager.exception;
-
-/**
- * Represents the exceptional case when a reconnection request is issued to a
- * node that cannot be reconnected (e.g., the node is not disconnected).
- *
- * @author unattributed
- */
-public class IllegalNodeReconnectionException extends IllegalClusterStateException {
-
-    public IllegalNodeReconnectionException() {
-    }
-
-    public IllegalNodeReconnectionException(String msg) {
-        super(msg);
-    }
-
-    public IllegalNodeReconnectionException(Throwable cause) {
-        super(cause);
-    }
-
-    public IllegalNodeReconnectionException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IneligiblePrimaryNodeException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IneligiblePrimaryNodeException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IneligiblePrimaryNodeException.java
deleted file mode 100644
index 4b0097a..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IneligiblePrimaryNodeException.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.cluster.manager.exception;
-
-/**
- * Represents the exceptional case when the primary role cannot be assigned to a
- * node because the node is ineligible for the role.
- *
- * @author unattributed
- */
-public class IneligiblePrimaryNodeException extends IllegalClusterStateException {
-
-    public IneligiblePrimaryNodeException() {
-    }
-
-    public IneligiblePrimaryNodeException(String msg) {
-        super(msg);
-    }
-
-    public IneligiblePrimaryNodeException(Throwable cause) {
-        super(cause);
-    }
-
-    public IneligiblePrimaryNodeException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/MutableRequestException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/MutableRequestException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/MutableRequestException.java
deleted file mode 100644
index d160587..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/MutableRequestException.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.cluster.manager.exception;
-
-/**
- * Represents the exceptional case when a HTTP request that may change a node's
- * state is to be replicated while the cluster or connected nodes are unable to
- * change their state (e.g., a new node is connecting to the cluster).
- *
- * @author unattributed
- */
-public class MutableRequestException extends IllegalClusterStateException {
-
-    public MutableRequestException() {
-    }
-
-    public MutableRequestException(String msg) {
-        super(msg);
-    }
-
-    public MutableRequestException(Throwable cause) {
-        super(cause);
-    }
-
-    public MutableRequestException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoConnectedNodesException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoConnectedNodesException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoConnectedNodesException.java
deleted file mode 100644
index 8d704b9..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoConnectedNodesException.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.cluster.manager.exception;
-
-/**
- * Represents the exceptional case when the cluster is unable to service a
- * request because no nodes are connected.
- *
- * @author unattributed
- */
-public class NoConnectedNodesException extends ClusterException {
-
-    public NoConnectedNodesException() {
-    }
-
-    public NoConnectedNodesException(String msg) {
-        super(msg);
-    }
-
-    public NoConnectedNodesException(Throwable cause) {
-        super(cause);
-    }
-
-    public NoConnectedNodesException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoResponseFromNodesException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoResponseFromNodesException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoResponseFromNodesException.java
deleted file mode 100644
index 9e17a23..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoResponseFromNodesException.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.cluster.manager.exception;
-
-/**
- * Represents the exceptional case when the cluster is unable to service a
- * request because no nodes returned a response. When the given request is not
- * mutable the nodes are left in their previous state.
- *
- * @author unattributed
- */
-public class NoResponseFromNodesException extends ClusterException {
-
-    public NoResponseFromNodesException() {
-    }
-
-    public NoResponseFromNodesException(String msg) {
-        super(msg);
-    }
-
-    public NoResponseFromNodesException(Throwable cause) {
-        super(cause);
-    }
-
-    public NoResponseFromNodesException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeDisconnectionException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeDisconnectionException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeDisconnectionException.java
deleted file mode 100644
index 3bd2f4b..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeDisconnectionException.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.cluster.manager.exception;
-
-/**
- * Represents the exceptional case when a disconnection request to a node
- * failed.
- *
- * @author unattributed
- */
-public class NodeDisconnectionException extends ClusterException {
-
-    public NodeDisconnectionException() {
-    }
-
-    public NodeDisconnectionException(String msg) {
-        super(msg);
-    }
-
-    public NodeDisconnectionException(Throwable cause) {
-        super(cause);
-    }
-
-    public NodeDisconnectionException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeReconnectionException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeReconnectionException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeReconnectionException.java
deleted file mode 100644
index 8c40cef..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeReconnectionException.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.cluster.manager.exception;
-
-/**
- * Represents the exceptional case when a reconnection request to a node failed.
- *
- * @author unattributed
- */
-public class NodeReconnectionException extends ClusterException {
-
-    public NodeReconnectionException() {
-    }
-
-    public NodeReconnectionException(String msg) {
-        super(msg);
-    }
-
-    public NodeReconnectionException(Throwable cause) {
-        super(cause);
-    }
-
-    public NodeReconnectionException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/PrimaryRoleAssignmentException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/PrimaryRoleAssignmentException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/PrimaryRoleAssignmentException.java
deleted file mode 100644
index 403f7a5..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/PrimaryRoleAssignmentException.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.cluster.manager.exception;
-
-/**
- * Represents the exceptional case when the cluster is unable to update the
- * primary role of a node.
- *
- * @author unattributed
- */
-public class PrimaryRoleAssignmentException extends IllegalClusterStateException {
-
-    public PrimaryRoleAssignmentException() {
-    }
-
-    public PrimaryRoleAssignmentException(String msg) {
-        super(msg);
-    }
-
-    public PrimaryRoleAssignmentException(Throwable cause) {
-        super(cause);
-    }
-
-    public PrimaryRoleAssignmentException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/SafeModeMutableRequestException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/SafeModeMutableRequestException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/SafeModeMutableRequestException.java
deleted file mode 100644
index f544f26..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/SafeModeMutableRequestException.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.cluster.manager.exception;
-
-/**
- * Represents the exceptional case when a HTTP request that may change a node's
- * dataflow is to be replicated while the cluster is in safe mode.
- *
- * @author unattributed
- */
-public class SafeModeMutableRequestException extends MutableRequestException {
-
-    public SafeModeMutableRequestException() {
-    }
-
-    public SafeModeMutableRequestException(String msg) {
-        super(msg);
-    }
-
-    public SafeModeMutableRequestException(Throwable cause) {
-        super(cause);
-    }
-
-    public SafeModeMutableRequestException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UnknownNodeException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UnknownNodeException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UnknownNodeException.java
deleted file mode 100644
index 914bb56..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UnknownNodeException.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.cluster.manager.exception;
-
-/**
- * Represents the exceptional case when a request is made for a node that does
- * not exist.
- *
- * @author unattributed
- */
-public class UnknownNodeException extends ClusterException {
-
-    public UnknownNodeException() {
-    }
-
-    public UnknownNodeException(String msg) {
-        super(msg);
-    }
-
-    public UnknownNodeException(Throwable cause) {
-        super(cause);
-    }
-
-    public UnknownNodeException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UriConstructionException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UriConstructionException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UriConstructionException.java
deleted file mode 100644
index 773d7b5..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UriConstructionException.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.cluster.manager.exception;
-
-/**
- * Represents the exceptional case when a URI cannot be constructed from the
- * given information. This exception is similar to Java's URISyntaxException
- * except that it extends RuntimeException.
- *
- * @author unattributed
- */
-public class UriConstructionException extends RuntimeException {
-
-    public UriConstructionException() {
-    }
-
-    public UriConstructionException(String msg) {
-        super(msg);
-    }
-
-    public UriConstructionException(Throwable cause) {
-        super(cause);
-    }
-
-    public UriConstructionException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java
deleted file mode 100644
index 2015530..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java
+++ /dev/null
@@ -1,135 +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.cluster.manager.impl;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.nifi.controller.status.ProcessGroupStatus;
-import org.apache.nifi.events.EventReporter;
-import org.apache.nifi.provenance.ProvenanceEventBuilder;
-import org.apache.nifi.provenance.ProvenanceEventRecord;
-import org.apache.nifi.provenance.ProvenanceEventRepository;
-import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
-import org.apache.nifi.provenance.search.Query;
-import org.apache.nifi.provenance.search.QuerySubmission;
-import org.apache.nifi.provenance.search.SearchableField;
-import org.apache.nifi.reporting.EventAccess;
-
-public class ClusteredEventAccess implements EventAccess {
-
-    private final WebClusterManager clusterManager;
-
-    public ClusteredEventAccess(final WebClusterManager clusterManager) {
-        this.clusterManager = clusterManager;
-    }
-
-    @Override
-    public ProcessGroupStatus getControllerStatus() {
-        return clusterManager.getProcessGroupStatus(WebClusterManager.ROOT_GROUP_ID_ALIAS);
-    }
-
-    @Override
-    public List<ProvenanceEventRecord> getProvenanceEvents(long arg0, int arg1) throws IOException {
-        return new ArrayList<>();
-    }
-
-    @Override
-    public ProvenanceEventRepository getProvenanceRepository() {
-        // NCM doesn't have provenance events, because it doesn't process FlowFiles.
-        // So we just use a Provenance Event Repository that does nothing.
-        return new ProvenanceEventRepository() {
-            @Override
-            public void close() throws IOException {
-            }
-
-            @Override
-            public ProvenanceEventRecord getEvent(long eventId) throws IOException {
-                return null;
-            }
-
-            @Override
-            public List<ProvenanceEventRecord> getEvents(long startEventId, int maxEvents) throws IOException {
-                return new ArrayList<>();
-            }
-
-            @Override
-            public Long getMaxEventId() {
-                return null;
-            }
-
-            @Override
-            public List<SearchableField> getSearchableAttributes() {
-                return new ArrayList<>();
-            }
-
-            @Override
-            public List<SearchableField> getSearchableFields() {
-                return new ArrayList<>();
-            }
-
-            @Override
-            public void registerEvent(final ProvenanceEventRecord event) {
-            }
-
-            @Override
-            public void registerEvents(final Iterable<ProvenanceEventRecord> events) {
-            }
-
-            @Override
-            public ComputeLineageSubmission retrieveLineageSubmission(final String submissionId) {
-                return null;
-            }
-
-            @Override
-            public QuerySubmission retrieveQuerySubmission(final String submissionId) {
-                return null;
-            }
-
-            @Override
-            public ComputeLineageSubmission submitExpandChildren(final long eventId) {
-                return null;
-            }
-
-            @Override
-            public ComputeLineageSubmission submitExpandParents(final long eventId) {
-                return null;
-            }
-
-            @Override
-            public ComputeLineageSubmission submitLineageComputation(final String flowFileUuid) {
-                return null;
-            }
-
-            @Override
-            public QuerySubmission submitQuery(final Query query) {
-                return null;
-            }
-
-            @Override
-            public ProvenanceEventBuilder eventBuilder() {
-                return null;
-            }
-
-            @Override
-            public void initialize(EventReporter eventReporter) throws IOException {
-
-            }
-        };
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredReportingContext.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredReportingContext.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredReportingContext.java
deleted file mode 100644
index e546f87..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredReportingContext.java
+++ /dev/null
@@ -1,165 +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.cluster.manager.impl;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.PreparedQuery;
-import org.apache.nifi.attribute.expression.language.Query;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.PropertyValue;
-import org.apache.nifi.controller.ControllerServiceLookup;
-import org.apache.nifi.controller.service.ControllerServiceProvider;
-import org.apache.nifi.controller.status.PortStatus;
-import org.apache.nifi.controller.status.ProcessGroupStatus;
-import org.apache.nifi.controller.status.ProcessorStatus;
-import org.apache.nifi.events.BulletinFactory;
-import org.apache.nifi.processor.StandardPropertyValue;
-import org.apache.nifi.reporting.Bulletin;
-import org.apache.nifi.reporting.BulletinRepository;
-import org.apache.nifi.reporting.EventAccess;
-import org.apache.nifi.reporting.ReportingContext;
-import org.apache.nifi.reporting.Severity;
-
-public class ClusteredReportingContext implements ReportingContext {
-
-    private final EventAccess eventAccess;
-    private final BulletinRepository bulletinRepository;
-    private final ControllerServiceProvider serviceProvider;
-    private final Map<PropertyDescriptor, String> properties;
-    private final Map<PropertyDescriptor, PreparedQuery> preparedQueries;
-
-    public ClusteredReportingContext(final EventAccess eventAccess, final BulletinRepository bulletinRepository,
-            final Map<PropertyDescriptor, String> properties, final ControllerServiceProvider serviceProvider) {
-        this.eventAccess = eventAccess;
-        this.bulletinRepository = bulletinRepository;
-        this.properties = Collections.unmodifiableMap(properties);
-        this.serviceProvider = serviceProvider;
-
-        preparedQueries = new HashMap<>();
-        for (final Map.Entry<PropertyDescriptor, String> entry : properties.entrySet()) {
-            final PropertyDescriptor desc = entry.getKey();
-            String value = entry.getValue();
-            if (value == null) {
-                value = desc.getDefaultValue();
-            }
-
-            final PreparedQuery pq = Query.prepare(value);
-            preparedQueries.put(desc, pq);
-        }
-    }
-
-    @Override
-    public EventAccess getEventAccess() {
-        return eventAccess;
-    }
-
-    @Override
-    public BulletinRepository getBulletinRepository() {
-        return bulletinRepository;
-    }
-
-    @Override
-    public Bulletin createBulletin(final String category, final Severity severity, final String message) {
-        return BulletinFactory.createBulletin(category, severity.name(), message);
-    }
-
-    @Override
-    public Bulletin createBulletin(final String componentId, final String category, final Severity severity, final String message) {
-        final ProcessGroupStatus rootGroupStatus = eventAccess.getControllerStatus();
-        final String groupId = findGroupId(rootGroupStatus, componentId);
-        final String componentName = findComponentName(rootGroupStatus, componentId);
-
-        return BulletinFactory.createBulletin(groupId, componentId, componentName, category, severity.name(), message);
-    }
-
-    @Override
-    public Map<PropertyDescriptor, String> getProperties() {
-        return Collections.unmodifiableMap(properties);
-    }
-
-    @Override
-    public PropertyValue getProperty(final PropertyDescriptor property) {
-        final String configuredValue = properties.get(property);
-        return new StandardPropertyValue(configuredValue == null ? property.getDefaultValue() : configuredValue, serviceProvider, preparedQueries.get(property));
-    }
-
-    @Override
-    public ControllerServiceLookup getControllerServiceLookup() {
-        return serviceProvider;
-    }
-
-    String findGroupId(final ProcessGroupStatus groupStatus, final String componentId) {
-        for (final ProcessorStatus procStatus : groupStatus.getProcessorStatus()) {
-            if (procStatus.getId().equals(componentId)) {
-                return groupStatus.getId();
-            }
-        }
-
-        for (final PortStatus portStatus : groupStatus.getInputPortStatus()) {
-            if (portStatus.getId().equals(componentId)) {
-                return groupStatus.getId();
-            }
-        }
-
-        for (final PortStatus portStatus : groupStatus.getOutputPortStatus()) {
-            if (portStatus.getId().equals(componentId)) {
-                return groupStatus.getId();
-            }
-        }
-
-        for (final ProcessGroupStatus childGroup : groupStatus.getProcessGroupStatus()) {
-            final String groupId = findGroupId(childGroup, componentId);
-            if (groupId != null) {
-                return groupId;
-            }
-        }
-
-        return null;
-    }
-
-    private String findComponentName(final ProcessGroupStatus groupStatus, final String componentId) {
-        for (final ProcessorStatus procStatus : groupStatus.getProcessorStatus()) {
-            if (procStatus.getId().equals(componentId)) {
-                return procStatus.getName();
-            }
-        }
-
-        for (final PortStatus portStatus : groupStatus.getInputPortStatus()) {
-            if (portStatus.getId().equals(componentId)) {
-                return groupStatus.getName();
-            }
-        }
-
-        for (final PortStatus portStatus : groupStatus.getOutputPortStatus()) {
-            if (portStatus.getId().equals(componentId)) {
-                return groupStatus.getName();
-            }
-        }
-
-        for (final ProcessGroupStatus childGroup : groupStatus.getProcessGroupStatus()) {
-            final String componentName = findComponentName(childGroup, componentId);
-            if (componentName != null) {
-                return componentName;
-            }
-        }
-
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java
deleted file mode 100644
index 81bb7a7..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java
+++ /dev/null
@@ -1,531 +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.cluster.manager.impl;
-
-import com.sun.jersey.api.client.Client;
-import com.sun.jersey.api.client.ClientResponse;
-import com.sun.jersey.api.client.UniformInterfaceException;
-import com.sun.jersey.api.client.WebResource;
-import com.sun.jersey.api.client.config.ClientConfig;
-import com.sun.jersey.api.client.filter.GZIPContentEncodingFilter;
-import com.sun.jersey.core.util.MultivaluedMapImpl;
-
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.Collection;
-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.Callable;
-import java.util.concurrent.CompletionService;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-
-import javax.ws.rs.HttpMethod;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.MultivaluedMap;
-
-import org.apache.nifi.cluster.manager.HttpRequestReplicator;
-import org.apache.nifi.cluster.manager.NodeResponse;
-import org.apache.nifi.cluster.manager.exception.UriConstructionException;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.logging.NiFiLog;
-import org.apache.nifi.util.FormatUtils;
-
-import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An implementation of the <code>HttpRequestReplicator</code> interface. This
- * implementation parallelizes the node HTTP requests using the given
- * <code>ExecutorService</code> instance. Individual requests may have
- * connection and read timeouts set, which may be set during instance
- * construction. Otherwise, the default is not to timeout.
- *
- * If a node protocol scheme is provided during construction, then all requests
- * will be replicated using the given scheme. If null is provided as the scheme
- * (the default), then the requests will be replicated using the scheme of the
- * original URI.
- *
- * Clients must call start() and stop() to initialize and shutdown the instance.
- * The instance must be started before issuing any replication requests.
- *
- * @author unattributed
- */
-public class HttpRequestReplicatorImpl implements HttpRequestReplicator {
-
-    // defaults
-    private static final int DEFAULT_SHUTDOWN_REPLICATOR_SECONDS = 30;
-
-    // logger
-    private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(HttpRequestReplicatorImpl.class));
-
-    // final members
-    private final Client client;            // the client to use for issuing requests
-    private final int numThreads;           // number of threads to use for request replication
-    private final int connectionTimeoutMs;  // connection timeout per node request
-    private final int readTimeoutMs;        // read timeout per node request
-
-    // members
-    private ExecutorService executorService;
-    private int shutdownReplicatorSeconds = DEFAULT_SHUTDOWN_REPLICATOR_SECONDS;
-
-    // guarded by synchronized method access in support of multithreaded replication
-    private String nodeProtocolScheme = null;
-
-    /**
-     * Creates an instance. The connection timeout and read timeout will be
-     * infinite.
-     *
-     * @param numThreads the number of threads to use when parallelizing
-     * requests
-     * @param client a client for making requests
-     */
-    public HttpRequestReplicatorImpl(final int numThreads, final Client client) {
-        this(numThreads, client, "0 sec", "0 sec");
-    }
-
-    /**
-     * Creates an instance.
-     *
-     * @param numThreads the number of threads to use when parallelizing
-     * requests
-     * @param client a client for making requests
-     * @param connectionTimeoutMs the connection timeout specified in
-     * milliseconds
-     * @param readTimeoutMs the read timeout specified in milliseconds
-     */
-    public HttpRequestReplicatorImpl(final int numThreads, final Client client, final String connectionTimeout, final String readTimeout) {
-
-        if (numThreads <= 0) {
-            throw new IllegalArgumentException("The number of threads must be greater than zero.");
-        } else if (client == null) {
-            throw new IllegalArgumentException("Client may not be null.");
-        }
-
-        this.numThreads = numThreads;
-        this.client = client;
-        this.connectionTimeoutMs = (int) FormatUtils.getTimeDuration(connectionTimeout, TimeUnit.MILLISECONDS);
-        this.readTimeoutMs = (int) FormatUtils.getTimeDuration(readTimeout, TimeUnit.MILLISECONDS);
-
-        client.getProperties().put(ClientConfig.PROPERTY_CONNECT_TIMEOUT, connectionTimeoutMs);
-        client.getProperties().put(ClientConfig.PROPERTY_READ_TIMEOUT, readTimeoutMs);
-        client.getProperties().put(ClientConfig.PROPERTY_FOLLOW_REDIRECTS, Boolean.TRUE);
-    }
-
-    @Override
-    public void start() {
-        if (isRunning()) {
-            throw new IllegalStateException("Instance is already started.");
-        }
-        executorService = Executors.newFixedThreadPool(numThreads);
-    }
-
-    @Override
-    public boolean isRunning() {
-        return executorService != null && !executorService.isShutdown();
-    }
-
-    @Override
-    public void stop() {
-
-        if (!isRunning()) {
-            throw new IllegalStateException("Instance is already stopped.");
-        }
-
-        // shutdown executor service
-        try {
-            if (getShutdownReplicatorSeconds() <= 0) {
-                executorService.shutdownNow();
-            } else {
-                executorService.shutdown();
-            }
-            executorService.awaitTermination(getShutdownReplicatorSeconds(), TimeUnit.SECONDS);
-        } catch (final InterruptedException ex) {
-            Thread.currentThread().interrupt();
-        } finally {
-            if (executorService.isTerminated()) {
-                logger.info("HTTP Request Replicator has been terminated successfully.");
-            } else {
-                logger.warn("HTTP Request Replicator has not terminated properly.  There exists an uninterruptable thread that will take an indeterminate amount of time to stop.");
-            }
-        }
-    }
-
-    /**
-     * Sets the protocol scheme to use when issuing requests to nodes.
-     *
-     * @param nodeProtocolScheme the scheme. Valid values are "http", "https",
-     * or null. If null is specified, then the scheme of the originating request
-     * is used when replicating that request.
-     */
-    public synchronized void setNodeProtocolScheme(final String nodeProtocolScheme) {
-        if (StringUtils.isNotBlank(nodeProtocolScheme)) {
-            if (!"http".equalsIgnoreCase(nodeProtocolScheme) && !"https".equalsIgnoreCase(nodeProtocolScheme)) {
-                throw new IllegalArgumentException("Node Protocol Scheme must be either HTTP or HTTPS");
-            }
-        }
-        this.nodeProtocolScheme = nodeProtocolScheme;
-    }
-
-    public synchronized String getNodeProtocolScheme() {
-        return nodeProtocolScheme;
-    }
-
-    private synchronized String getNodeProtocolScheme(final URI uri) {
-        // if we are not configured to use a protocol scheme, then use the uri's scheme
-        if (StringUtils.isBlank(nodeProtocolScheme)) {
-            return uri.getScheme();
-        }
-        return nodeProtocolScheme;
-    }
-
-    public int getConnectionTimeoutMs() {
-        return connectionTimeoutMs;
-    }
-
-    public int getReadTimeoutMs() {
-        return readTimeoutMs;
-    }
-
-    public int getShutdownReplicatorSeconds() {
-        return shutdownReplicatorSeconds;
-    }
-
-    public void setShutdownReplicatorSeconds(int shutdownReplicatorSeconds) {
-        this.shutdownReplicatorSeconds = shutdownReplicatorSeconds;
-    }
-
-    @Override
-    public Set<NodeResponse> replicate(final Set<NodeIdentifier> nodeIds, final String method,
-            final URI uri, final Map<String, List<String>> parameters, final Map<String, String> headers)
-            throws UriConstructionException {
-        if (nodeIds == null) {
-            throw new IllegalArgumentException("Node IDs may not be null.");
-        } else if (method == null) {
-            throw new IllegalArgumentException("HTTP method may not be null.");
-        } else if (uri == null) {
-            throw new IllegalArgumentException("URI may not be null.");
-        } else if (parameters == null) {
-            throw new IllegalArgumentException("Parameters may not be null.");
-        } else if (headers == null) {
-            throw new IllegalArgumentException("HTTP headers map may not be null.");
-        }
-        return replicateHelper(nodeIds, method, getNodeProtocolScheme(uri), uri.getPath(), parameters, /* entity */ null, headers);
-    }
-
-    @Override
-    public Set<NodeResponse> replicate(final Set<NodeIdentifier> nodeIds, final String method, final URI uri,
-            final Object entity, final Map<String, String> headers) throws UriConstructionException {
-        if (nodeIds == null) {
-            throw new IllegalArgumentException("Node IDs may not be null.");
-        } else if (method == null) {
-            throw new IllegalArgumentException("HTTP method may not be null.");
-        } else if (method.equalsIgnoreCase(HttpMethod.DELETE) || method.equalsIgnoreCase(HttpMethod.GET) || method.equalsIgnoreCase(HttpMethod.HEAD) || method.equalsIgnoreCase(HttpMethod.OPTIONS)) {
-            throw new IllegalArgumentException("HTTP (DELETE | GET | HEAD | OPTIONS) requests cannot have a body containing an entity.");
-        } else if (uri == null) {
-            throw new IllegalArgumentException("URI may not be null.");
-        } else if (entity == null) {
-            throw new IllegalArgumentException("Entity may not be null.");
-        } else if (headers == null) {
-            throw new IllegalArgumentException("HTTP headers map may not be null.");
-        }
-        return replicateHelper(nodeIds, method, getNodeProtocolScheme(uri), uri.getPath(), /* parameters */ null, entity, headers);
-    }
-
-    private Set<NodeResponse> replicateHelper(final Set<NodeIdentifier> nodeIds, final String method, final String scheme,
-            final String path, final Map<String, List<String>> parameters, final Object entity, final Map<String, String> headers)
-            throws UriConstructionException {
-
-        if (nodeIds.isEmpty()) {
-            return new HashSet<>(); // return quickly for trivial case
-        }
-
-        final CompletionService<NodeResponse> completionService = new ExecutorCompletionService<>(executorService);
-
-        // keeps track of future requests so that failed requests can be tied back to the failing node
-        final Collection<NodeHttpRequestFutureWrapper> futureNodeHttpRequests = new ArrayList<>();
-
-        // construct the URIs for the nodes
-        final Map<NodeIdentifier, URI> uriMap = new HashMap<>();
-        try {
-            for (final NodeIdentifier nodeId : nodeIds) {
-                final URI nodeUri = new URI(scheme, null, nodeId.getApiAddress(), nodeId.getApiPort(), path, /* query */ null, /* fragment */ null);
-                uriMap.put(nodeId, nodeUri);
-            }
-        } catch (final URISyntaxException use) {
-            throw new UriConstructionException(use);
-        }
-
-        // submit the requests to the nodes
-        final String requestId = UUID.randomUUID().toString();
-        headers.put(WebClusterManager.REQUEST_ID_HEADER, requestId);
-        for (final Map.Entry<NodeIdentifier, URI> entry : uriMap.entrySet()) {
-            final NodeIdentifier nodeId = entry.getKey();
-            final URI nodeUri = entry.getValue();
-            final NodeHttpRequestCallable callable = (entity == null)
-                    ? new NodeHttpRequestCallable(nodeId, method, nodeUri, parameters, headers)
-                    : new NodeHttpRequestCallable(nodeId, method, nodeUri, entity, headers);
-            futureNodeHttpRequests.add(new NodeHttpRequestFutureWrapper(nodeId, method, nodeUri, completionService.submit(callable)));
-        }
-
-        // get the node responses
-        final Set<NodeResponse> result = new HashSet<>();
-        for (int i = 0; i < nodeIds.size(); i++) {
-
-            // keeps track of the original request information in case we receive an exception
-            NodeHttpRequestFutureWrapper futureNodeHttpRequest = null;
-            try {
-
-                // get the future resource response for the node
-                final Future<NodeResponse> futureNodeResourceResponse = completionService.take();
-
-                // find the original request by comparing the submitted future with the future returned by the completion service
-                for (final NodeHttpRequestFutureWrapper futureNodeHttpRequestElem : futureNodeHttpRequests) {
-                    if (futureNodeHttpRequestElem.getFuture() == futureNodeResourceResponse) {
-                        futureNodeHttpRequest = futureNodeHttpRequestElem;
-                    }
-                }
-
-                // try to retrieve the node response and add to result
-                final NodeResponse nodeResponse = futureNodeResourceResponse.get();
-                result.add(nodeResponse);
-
-            } catch (final InterruptedException | ExecutionException ex) {
-
-                logger.warn("Node request for " + futureNodeHttpRequest.getNodeId() + " encountered exception: " + ex, ex);
-
-                // create node response with the thrown exception and add to result
-                final NodeResponse nodeResponse = new NodeResponse(
-                        futureNodeHttpRequest.getNodeId(), futureNodeHttpRequest.getHttpMethod(), futureNodeHttpRequest.getRequestUri(), ex);
-                result.add(nodeResponse);
-
-            }
-        }
-
-        if (logger.isDebugEnabled()) {
-            NodeResponse min = null;
-            NodeResponse max = null;
-            long nanosSum = 0L;
-            int nanosAdded = 0;
-
-            for (final NodeResponse response : result) {
-                final long requestNanos = response.getRequestDuration(TimeUnit.NANOSECONDS);
-                final long minNanos = (min == null) ? -1 : min.getRequestDuration(TimeUnit.NANOSECONDS);
-                final long maxNanos = (max == null) ? -1 : max.getRequestDuration(TimeUnit.NANOSECONDS);
-
-                if (requestNanos < minNanos || minNanos < 0L) {
-                    min = response;
-                }
-
-                if (requestNanos > maxNanos || maxNanos < 0L) {
-                    max = response;
-                }
-
-                if (requestNanos >= 0L) {
-                    nanosSum += requestNanos;
-                    nanosAdded++;
-                }
-            }
-
-            final StringBuilder sb = new StringBuilder();
-            sb.append("Node Responses for ").append(method).append(" ").append(path).append(" (Request ID ").append(requestId).append("):\n");
-            for (final NodeResponse response : result) {
-                sb.append(response).append("\n");
-            }
-
-            final long averageNanos = (nanosAdded == 0) ? -1L : nanosSum / nanosAdded;
-            final long averageMillis = (averageNanos < 0) ? averageNanos : TimeUnit.MILLISECONDS.convert(averageNanos, TimeUnit.NANOSECONDS);
-            logger.debug("For {} {} (Request ID {}), minimum response time = {}, max = {}, average = {} ms",
-                    method, path, requestId, min, max, averageMillis);
-            logger.debug(sb.toString());
-        }
-
-        return result;
-    }
-
-    /**
-     * Wraps a future node response with info from originating request. This
-     * coupling allows for futures that encountered exceptions to be linked back
-     * to the failing node and better reported.
-     */
-    private class NodeHttpRequestFutureWrapper {
-
-        private final NodeIdentifier nodeId;
-
-        private final String httpMethod;
-
-        private final URI requestUri;
-
-        private final Future<NodeResponse> future;
-
-        public NodeHttpRequestFutureWrapper(final NodeIdentifier nodeId, final String httpMethod,
-                final URI requestUri, final Future<NodeResponse> future) {
-            if (nodeId == null) {
-                throw new IllegalArgumentException("Node ID may not be null.");
-            } else if (StringUtils.isBlank(httpMethod)) {
-                throw new IllegalArgumentException("Http method may not be null or empty.");
-            } else if (requestUri == null) {
-                throw new IllegalArgumentException("Request URI may not be null.");
-            } else if (future == null) {
-                throw new IllegalArgumentException("Future may not be null.");
-            }
-            this.nodeId = nodeId;
-            this.httpMethod = httpMethod;
-            this.requestUri = requestUri;
-            this.future = future;
-        }
-
-        public NodeIdentifier getNodeId() {
-            return nodeId;
-        }
-
-        public String getHttpMethod() {
-            return httpMethod;
-        }
-
-        public URI getRequestUri() {
-            return requestUri;
-        }
-
-        public Future<NodeResponse> getFuture() {
-            return future;
-        }
-    }
-
-    /**
-     * A Callable for making an HTTP request to a single node and returning its
-     * response.
-     */
-    private class NodeHttpRequestCallable implements Callable<NodeResponse> {
-
-        private final NodeIdentifier nodeId;
-        private final String method;
-        private final URI uri;
-        private final Object entity;
-        private final Map<String, List<String>> parameters = new HashMap<>();
-        private final Map<String, String> headers = new HashMap<>();
-
-        private NodeHttpRequestCallable(final NodeIdentifier nodeId, final String method,
-                final URI uri, final Object entity, final Map<String, String> headers) {
-            this.nodeId = nodeId;
-            this.method = method;
-            this.uri = uri;
-            this.entity = entity;
-            this.headers.putAll(headers);
-        }
-
-        private NodeHttpRequestCallable(final NodeIdentifier nodeId, final String method,
-                final URI uri, final Map<String, List<String>> parameters, final Map<String, String> headers) {
-            this.nodeId = nodeId;
-            this.method = method;
-            this.uri = uri;
-            this.entity = null;
-            this.parameters.putAll(parameters);
-            this.headers.putAll(headers);
-        }
-
-        @Override
-        public NodeResponse call() {
-
-            try {
-                // create and send the request
-                final WebResource.Builder resourceBuilder = getResourceBuilder();
-                final String requestId = headers.get("x-nifi-request-id");
-
-                final long startNanos = System.nanoTime();
-                final ClientResponse clientResponse;
-                if (HttpMethod.DELETE.equalsIgnoreCase(method)) {
-                    clientResponse = resourceBuilder.delete(ClientResponse.class);
-                } else if (HttpMethod.GET.equalsIgnoreCase(method)) {
-                    clientResponse = resourceBuilder.get(ClientResponse.class);
-                } else if (HttpMethod.HEAD.equalsIgnoreCase(method)) {
-                    clientResponse = resourceBuilder.head();
-                } else if (HttpMethod.OPTIONS.equalsIgnoreCase(method)) {
-                    clientResponse = resourceBuilder.options(ClientResponse.class);
-                } else if (HttpMethod.POST.equalsIgnoreCase(method)) {
-                    clientResponse = resourceBuilder.post(ClientResponse.class);
-                } else if (HttpMethod.PUT.equalsIgnoreCase(method)) {
-                    clientResponse = resourceBuilder.put(ClientResponse.class);
-                } else {
-                    throw new IllegalArgumentException("HTTP Method '" + method + "' not supported for request replication.");
-                }
-
-                // create and return the response
-                return new NodeResponse(nodeId, method, uri, clientResponse, System.nanoTime() - startNanos, requestId);
-
-            } catch (final UniformInterfaceException | IllegalArgumentException t) {
-                return new NodeResponse(nodeId, method, uri, t);
-            }
-
-        }
-
-        private WebResource.Builder getResourceBuilder() {
-
-            // convert parameters to a more convenient data structure
-            final MultivaluedMap<String, String> map = new MultivaluedMapImpl();
-            map.putAll(parameters);
-
-            // create the resource
-            WebResource resource = client.resource(uri);
-
-            if (WebClusterManager.isResponseInterpreted(uri, method)) {
-                resource.addFilter(new GZIPContentEncodingFilter(false));
-            }
-
-            // set the parameters as either query parameters or as request body
-            final WebResource.Builder builder;
-            if (HttpMethod.DELETE.equalsIgnoreCase(method) || HttpMethod.HEAD.equalsIgnoreCase(method) || HttpMethod.GET.equalsIgnoreCase(method) || HttpMethod.OPTIONS.equalsIgnoreCase(method)) {
-                resource = resource.queryParams(map);
-                builder = resource.getRequestBuilder();
-            } else {
-                if (entity == null) {
-                    builder = resource.entity(map);
-                } else {
-                    builder = resource.entity(entity);
-                }
-            }
-
-            // set headers
-            boolean foundContentType = false;
-            for (final Map.Entry<String, String> entry : headers.entrySet()) {
-                builder.header(entry.getKey(), entry.getValue());
-                if (entry.getKey().equalsIgnoreCase("content-type")) {
-                    foundContentType = true;
-                }
-            }
-
-            // set default content type
-            if (!foundContentType) {
-                // set default content type
-                builder.type(MediaType.APPLICATION_FORM_URLENCODED);
-            }
-
-            return builder;
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java
deleted file mode 100644
index afade7e..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java
+++ /dev/null
@@ -1,85 +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.cluster.manager.impl;
-
-import java.net.URI;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import org.apache.nifi.cluster.manager.HttpResponseMapper;
-import org.apache.nifi.cluster.manager.NodeResponse;
-import org.apache.nifi.cluster.node.Node;
-import org.apache.nifi.cluster.node.Node.Status;
-import org.apache.nifi.logging.NiFiLog;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Determines the status of nodes based on their HTTP response codes.
- *
- * The algorithm is as follows.
- *
- * If any HTTP responses were 2XX, then disconnect non-2XX responses. This is
- * because 2XX may have changed a node's flow.
- *
- * If no 2XX responses were received, then the node's flow has not changed.
- * Instead of disconnecting everything, we only disconnect the nodes with
- * internal errors, i.e., 5XX responses.
- *
- * @author unattributed
- */
-public class HttpResponseMapperImpl implements HttpResponseMapper {
-
-    private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(HttpResponseMapperImpl.class));
-
-    @Override
-    public Map<NodeResponse, Status> map(final URI requestURI, final Set<NodeResponse> nodeResponses) {
-
-        final Map<NodeResponse, Status> result = new HashMap<>();
-
-        // check if any responses were 2XX
-        boolean found2xx = false;
-        for (final NodeResponse nodeResponse : nodeResponses) {
-            if (nodeResponse.is2xx()) {
-                found2xx = true;
-                break;
-            }
-        }
-
-        // determine the status of each node 
-        for (final NodeResponse nodeResponse : nodeResponses) {
-
-            final Node.Status status;
-            if (found2xx) {
-                // disconnect nodes with non-2XX responses
-                status = nodeResponse.is2xx()
-                        ? Node.Status.CONNECTED
-                        : Node.Status.DISCONNECTED;
-            } else {
-                // disconnect nodes with 5XX responses or exception
-                status = nodeResponse.is5xx()
-                        ? Node.Status.DISCONNECTED
-                        : Node.Status.CONNECTED;
-            }
-
-            result.put(nodeResponse, status);
-        }
-
-        return result;
-    }
-
-}


[32/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteScript.java
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteScript.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteScript.java
deleted file mode 100644
index 3a34769..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteScript.java
+++ /dev/null
@@ -1,939 +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.script;
-
-import org.apache.nifi.processors.script.ExecuteScript;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.util.MockFlowFile;
-import org.apache.nifi.util.TestRunner;
-import org.apache.nifi.util.TestRunners;
-
-import org.apache.commons.io.FileUtils;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author unattributed
- *
- */
-public class TestExecuteScript {
-
-    static Logger LOG;
-
-    static {
-        System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
-        System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
-        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.script.ExecuteScript", "trace");
-        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.script.TestExecuteScript", "debug");
-        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.AbstractProcessor", "debug");
-        LOG = LoggerFactory.getLogger(TestExecuteScript.class);
-    }
-
-    private TestRunner controller;
-
-    private final String multiline = "Lorem ipsum dolor sit amet,\n"
-            + "consectetur adipisicing elit,\n"
-            + "sed do eiusmod tempor incididunt ut labore et dolore magna aliqua.\n"
-            + "Ut enim ad minim veniam,\n"
-            + "quis nostrud exercitation ullamco laboris nisi ut aliquip ex ea commodo consequat.\n"
-            + "Duis aute irure dolor in reprehenderit in voluptate velit esse cillum dolore eu fugiat nulla pariatur.\n"
-            + "Excepteur sint occaecat cupidatat non proident,\n"
-            + "sunt in culpa qui officia deserunt mollit anim id est laborum.";
-
-    /**
-     * Create a mock SingleProcessorController using our processor and pass data
-     * to it via byte array. Returns the Sink that provides access to any files
-     * that pass out of the processor
-     */
-    @Before
-    public void setupEach() throws IOException {
-        controller = TestRunners.newTestRunner(ExecuteScript.class);
-        controller.setValidateExpressionUsage(false);
-
-        // copy all scripts to target directory and run from there. some python
-        // scripts create .class files that end up in src/test/resources.
-        FileUtils.copyDirectory(new File("src/test/resources"), new File("target/test-scripts"));
-    }
-
-    // Fail if the specified relationship does not contain exactly one file
-    // with the expected value
-    private void assertRelationshipContents(String expected, String relationship) {
-        controller.assertTransferCount(relationship, 1);
-        MockFlowFile ff = controller.getFlowFilesForRelationship(relationship).get(0);
-        ff.assertContentEquals(expected);
-    }
-
-    // Fail if the specified relationship does not contain specified number of files
-    // with the expected value
-    private void assertRelationshipContents(String expected, String relationship, int count) {
-        controller.assertTransferCount(relationship, count);
-        MockFlowFile ff = controller.getFlowFilesForRelationship(relationship).get(count - 1);
-        ff.assertContentEquals(expected);
-    }
-
-    // ////////////////////////////////////
-    // General tests
-    @Test(expected = IllegalArgumentException.class)
-    public void failOnBadName() {
-        LOG.info("Supplying bad script file names");
-
-        // None of these should result in actually setting the property, because they're non-existent / bad files
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "not/really.rb");
-        controller.assertNotValid();
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "fakey/fake.js");
-        controller.assertNotValid();
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "pom.xml");
-        controller.assertNotValid();
-    }
-
-    // ////////////////////////////////////
-    // Ruby script tests
-    @Test
-    public void testSimpleReadR() {
-        LOG.info("Ruby script: fail file based on reading contents");
-
-        controller.enqueue("This stuff is fine".getBytes());
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readTest.rb");
-        controller.setThreadCount(2);
-        controller.run(2);
-
-        assertRelationshipContents(multiline, "failure");
-        assertRelationshipContents("This stuff is fine", "success");
-
-        controller.getFlowFilesForRelationship("success").get(0).assertAttributeEquals("filename", "NewFileNameFromReadTest");
-    }
-
-    @Test
-    public void testParamReadR() {
-        LOG.info("Ruby script: Failing file based on reading contents");
-
-        Map<String, String> attrs1 = new HashMap<>();
-        attrs1.put("filename", "StuffIsFine.txt");
-        Map<String, String> attrs2 = new HashMap<>();
-        attrs2.put("filename", "multiline.txt");
-        controller.enqueue("This stuff is fine".getBytes(), attrs1);
-        controller.enqueue(multiline.getBytes(), attrs2);
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.rb");
-        controller.setProperty("expr", "rehenderit");
-        controller.run(2);
-
-        assertRelationshipContents(multiline, "failure");
-        assertRelationshipContents("This stuff is fine", "success");
-    }
-
-    @Test
-    public void testWriteLastLineR() {
-        LOG.info("Running Ruby script to output last line of file");
-
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/writeTest.rb");
-        controller.run();
-
-        List<MockFlowFile> files = controller.getFlowFilesForRelationship("success");
-
-        assertEquals("Process did not generate an output file", 1, files.size());
-
-        byte[] blob = files.get(0).toByteArray();
-        String[] lines = new String(blob).split("\n");
-
-        assertEquals("File had more than one line", 1, lines.length);
-        assertEquals("sunt in culpa qui officia deserunt mollit anim id est laborum.", lines[0]);
-    }
-
-    @Test
-    public void testWriteOptionalParametersR() {
-        LOG.info("Ruby script that uses optional parameters");
-
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/paramTest.rb");
-        controller.setProperty("repeat", "3");
-        controller.run();
-
-        List<MockFlowFile> files = controller.getFlowFilesForRelationship("success");
-
-        assertEquals("Process did not generate an output file", 1, files.size());
-
-        byte[] blob = files.get(0).toByteArray();
-        String[] lines = new String(blob).split("\n");
-
-        assertEquals("File did not have 3 lines", 3, lines.length);
-        assertEquals("sunt in culpa qui officia deserunt mollit anim id est laborum.", lines[0]);
-    }
-
-    @Test
-    public void testSetupOptionalValidationR() {
-        LOG.info("Ruby script creating validators for optional properties");
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/optionalValidators.rb");
-        controller.assertNotValid();
-        controller.setProperty("int", "abc");
-        controller.assertNotValid();
-        controller.setProperty("url", "not@valid");
-        controller.assertNotValid();
-        controller.setProperty("nonEmpty", "");
-        controller.assertNotValid();
-
-        controller.setProperty("int", "123");
-        controller.setProperty("url", "http://localhost");
-        controller.setProperty("nonEmpty", "abc123");
-        controller.assertValid();
-    }
-
-    @Test
-    public void testTwoScriptsSameThreadSameClassName() {
-        LOG.info("Test 2 different scripts with the same ruby class name");
-
-        Map<String, String> attrs1 = new HashMap<>();
-        attrs1.put("filename", "StuffIsFine.txt");
-        Map<String, String> attrs2 = new HashMap<>();
-        attrs2.put("filename", "multiline.txt");
-
-        controller.enqueue("This stuff is fine".getBytes(), attrs1);
-        controller.enqueue(multiline.getBytes(), attrs2);
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.rb");
-        controller.setProperty("expr", "rehenderit");
-        controller.run(2);
-
-        assertRelationshipContents(multiline, "failure");
-        assertRelationshipContents("This stuff is fine", "success");
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/optionalValidators.rb");
-        controller.assertNotValid();
-        controller.setProperty("int", "abc");
-        controller.assertNotValid();
-        controller.setProperty("url", "not@valid");
-        controller.assertNotValid();
-        controller.setProperty("nonEmpty", "");
-        controller.assertNotValid();
-
-        controller.setProperty("int", "123");
-        controller.setProperty("url", "http://localhost");
-        controller.setProperty("nonEmpty", "abc123");
-        controller.assertValid();
-    }
-
-    @Test
-    public void testUpdateScriptR() throws Exception {
-        LOG.info("Test one script with updated class");
-
-        File testFile = File.createTempFile("script", ".rb");
-        File original = new File("target/test-scripts/readWithParams.rb");
-        FileUtils.copyFile(original, testFile);
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, testFile.getPath());
-        controller.assertValid();
-        original = new File("target/test-scripts/optionalValidators.rb");
-        FileUtils.copyFile(original, testFile);
-        controller.setProperty(ExecuteScript.SCRIPT_CHECK_INTERVAL, "5 secs");
-        Thread.sleep(6000);
-
-        controller.assertNotValid();
-        controller.setProperty("int", "abc");
-        controller.assertNotValid();
-        controller.setProperty("url", "not@valid");
-        controller.assertNotValid();
-        controller.setProperty("nonEmpty", "");
-        controller.assertNotValid();
-
-        controller.setProperty("int", "123");
-        controller.setProperty("url", "http://localhost");
-        controller.setProperty("nonEmpty", "abc123");
-        controller.assertValid();
-        FileUtils.deleteQuietly(testFile);
-    }
-
-    @Test
-    public void testMultiThreadExecR() {
-        LOG.info("Ruby script 20 threads: Failing file based on reading contents");
-
-        Map<String, String> attrs1 = new HashMap<>();
-        attrs1.put("filename", "StuffIsFine.txt");
-        Map<String, String> attrs2 = new HashMap<>();
-        attrs2.put("filename", "multiline.txt");
-        controller.setThreadCount(20);
-        for (int i = 0; i < 10; i++) {
-            controller.enqueue("This stuff is fine".getBytes(), attrs1);
-            controller.enqueue(multiline.getBytes(), attrs2);
-        }
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.rb");
-        controller.setProperty("expr", "rehenderit");
-        controller.run(20);
-
-        controller.assertTransferCount("failure", 10);
-        controller.assertTransferCount("success", 10);
-        for (int i = 0; i < 10; i++) {
-            MockFlowFile ff = controller.getFlowFilesForRelationship("failure").get(i);
-            ff.assertContentEquals(multiline);
-            assertTrue(ff.getAttribute("filename").endsWith("modified"));
-            ff = controller.getFlowFilesForRelationship("success").get(i);
-            ff.assertContentEquals("This stuff is fine");
-            assertTrue(ff.getAttribute("filename").endsWith("modified"));
-        }
-
-    }
-
-    @Test
-    public void testManualValidationR() {
-        LOG.info("Ruby script defining manual validator");
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/alwaysFail.rb");
-        controller.assertNotValid();
-    }
-
-    @Test
-    public void testGetRelationshipsR() {
-        LOG.info("Ruby script: getRelationships");
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/routeTest.rb");
-        // at this point, the script has not been instantiated so the processor simply returns an empty set
-        Set<Relationship> rels = controller.getProcessor().getRelationships();
-        assertEquals(0, rels.size());
-        // this will instantiate the script
-        controller.assertValid();
-        // this will call the script
-        rels = controller.getProcessor().getRelationships();
-        assertEquals(3, rels.size());
-    }
-
-    @Test
-    public void testGetExceptionRouteR() {
-        LOG.info("Ruby script defining route taken in event of exception");
-
-        controller.enqueue("This stuff is fine".getBytes());
-        controller.enqueue("Bad things go to 'b'.".getBytes());
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/routeTest.rb");
-
-        controller.run(3);
-
-        assertRelationshipContents("This stuff is fine", "a");
-        assertRelationshipContents("Bad things go to 'b'.", "b");
-        assertRelationshipContents(multiline, "c");
-
-    }
-
-    @Test
-    public void testSimpleConverterR() {
-        LOG.info("Running Ruby converter script");
-
-        for (int i = 0; i < 20; i++) {
-            controller.enqueue(multiline.getBytes());
-        }
-
-        controller.setThreadCount(20);
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/simpleConverter.rb");
-        controller.run(20);
-
-        List<MockFlowFile> successFiles = controller.getFlowFilesForRelationship("success");
-        List<MockFlowFile> failFiles = controller.getFlowFilesForRelationship("failure");
-
-        assertEquals("Process did not generate 20 SUCCESS files", 20, successFiles.size());
-        assertEquals("Process did not generate 20 FAILURE files", 20, failFiles.size());
-
-        MockFlowFile sFile = successFiles.get(19);
-        MockFlowFile fFile = failFiles.get(19);
-
-        byte[] blob = fFile.toByteArray();
-        String[] lines = new String(blob).split("\n");
-
-        assertEquals("File had more than one line", 1, lines.length);
-        assertEquals("Lorem ipsum dolor sit amet,", lines[0]);
-
-        blob = sFile.toByteArray();
-        lines = new String(blob).split("\n");
-
-        assertEquals("SUCCESS had wrong number of lines", 7, lines.length);
-        assertEquals("consectetur adipisicing elit,", lines[0]);
-    }
-
-    @Test
-    public void testLoadLocalR() {
-        LOG.info("Ruby: load another script file");
-
-        controller.enqueue("This stuff is fine".getBytes());
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/loadLocal.rb");
-        controller.run(2);
-
-        assertRelationshipContents(multiline, "failure");
-        assertRelationshipContents("This stuff is fine", "success");
-    }
-
-    @Test
-    public void testFlowFileR() {
-        LOG.info("Ruby: get FlowFile properties");
-
-        controller.enqueue(multiline.getBytes());
-        HashMap<String, String> meta = new HashMap<String, String>();
-        meta.put("evict", "yup");
-        controller.enqueue("This would be plenty long but it's also evicted.".getBytes(), meta);
-        controller.enqueue("This is too short".getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/ffTest.rb");
-        controller.run(3);
-
-        assertRelationshipContents(multiline, "success");
-        assertRelationshipContents("This is too short", "failure");
-        assertRelationshipContents("This would be plenty long but it's also evicted.", "evict");
-    }
-
-    // //////////////////////////////////// // JS tests
-    @Test
-    public void testSimpleReadJS() {
-        LOG.info("Javascript: fail file based on reading contents");
-
-        controller.enqueue("This stuff is fine".getBytes());
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readTest.js");
-        controller.run(2);
-
-        assertRelationshipContents(multiline, "failure");
-        assertRelationshipContents("This stuff is fine", "success");
-    }
-
-    @Test
-    public void testParamReadJS() {
-        LOG.info("Javascript: read contents and fail based on parameter");
-
-        controller.enqueue("This stuff is fine".getBytes());
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.js");
-        controller.setProperty("expr", "sed do");
-        controller.run(2);
-
-        assertRelationshipContents(multiline, "failure");
-        assertRelationshipContents("This stuff is fine", "success");
-    }
-
-    @Test
-    public void testWriteLastLineJS() {
-        LOG.info("Running Javascript to output last line of file");
-
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/writeTest.js");
-        controller.run();
-
-        List<MockFlowFile> sunkFiles = controller.getFlowFilesForRelationship("success");
-
-        assertEquals("Process did not generate an output file", 1, sunkFiles.size());
-
-        MockFlowFile sunkFile = sunkFiles.iterator().next();
-        byte[] blob = sunkFile.toByteArray();
-        String[] lines = new String(blob).split("\n");
-
-        assertEquals("File had more than one line", 1, lines.length);
-        assertEquals("sunt in culpa qui officia deserunt mollit anim id est laborum.", lines[0]);
-    }
-
-    @Test
-    public void testWriteOptionalParametersJS() {
-        LOG.info("Javascript processCallback that uses optional parameters");
-
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/paramTest.js");
-        controller.setProperty("repeat", "3");
-        controller.run();
-
-        List<MockFlowFile> sunkFiles = controller.getFlowFilesForRelationship("success");
-
-        assertEquals("Process did not generate an output file", 1, sunkFiles.size());
-
-        MockFlowFile sunkFile = sunkFiles.iterator().next();
-        byte[] blob = sunkFile.toByteArray();
-        String[] lines = new String(blob).split("\n");
-
-        assertEquals("File did not have 3 lines", 3, lines.length);
-        assertEquals("sunt in culpa qui officia deserunt mollit anim id est laborum.", lines[0]);
-    }
-
-    @Test
-    public void testSetupOptionalValidationJS() {
-        LOG.info("Javascript creating validators for optional properties");
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/optionalValidators.js");
-        controller.setProperty("int", "abc");
-        controller.setProperty("url", "not@valid");
-        controller.setProperty("nonEmpty", "");
-        assertEquals(2, controller.getProcessor().getPropertyDescriptors().size());
-        controller.assertNotValid(); // due to invalid values above
-        assertEquals(5, controller.getProcessor().getPropertyDescriptors().size());
-
-        controller.setProperty("int", "123");
-        controller.setProperty("url", "http://localhost");
-        controller.setProperty("nonEmpty", "abc123");
-        assertEquals(5, controller.getProcessor().getPropertyDescriptors().size());
-        controller.assertValid();
-    }
-
-    @Test
-    public void testManualValidationJS() {
-        LOG.info("Javascript defining manual validator");
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/alwaysFail.js");
-        controller.assertNotValid();
-    }
-
-    @Test
-    public void testGetExceptionRouteJS() {
-        LOG.info("Javascript defining route taken in event of exception");
-
-        controller.enqueue("This stuff is fine".getBytes());
-        controller.enqueue("Bad things go to 'b'.".getBytes());
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/routeTest.js");
-
-        controller.run(3);
-
-        assertRelationshipContents("This stuff is fine", "a");
-        assertRelationshipContents("Bad things go to 'b'.", "b");
-        assertRelationshipContents(multiline, "c");
-
-    }
-
-    @Test
-    public void testSimpleConverterJS() {
-        LOG.info("Running Javascript converter script");
-
-        for (int i = 0; i < 20; i++) {
-            controller.enqueue(multiline.getBytes());
-        }
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/simpleConverter.js");
-        controller.run(20);
-
-        List<MockFlowFile> successFiles = controller.getFlowFilesForRelationship("success");
-        List<MockFlowFile> failFiles = controller.getFlowFilesForRelationship("failure");
-
-        assertEquals("Process did not generate 20 SUCCESS files", 20, successFiles.size());
-        assertEquals("Process did not generate 20 FAILURE file", 20, failFiles.size());
-
-        MockFlowFile sFile = successFiles.get(19);
-        MockFlowFile fFile = failFiles.get(0);
-
-        byte[] blob = sFile.toByteArray();
-        String[] lines = new String(blob).split("\n");
-
-        assertEquals("SUCCESS had wrong number of lines", 7, lines.length);
-        assertTrue(lines[0].startsWith("consectetur adipisicing elit,"));
-
-        blob = fFile.toByteArray();
-        lines = new String(blob).split("\n");
-
-        assertEquals("File had more than one line", 1, lines.length);
-        assertTrue(lines[0].startsWith("Lorem ipsum dolor sit amet,"));
-    }
-
-    @Test
-    public void testLoadLocalJS() {
-        LOG.info("Javascript: load another script file");
-
-        controller.enqueue("This stuff is fine".getBytes());
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/loadLocal.js");
-        controller.run(2);
-
-        assertRelationshipContents(multiline, "failure");
-        assertRelationshipContents("This stuff is fine", "success");
-    }
-
-    @Test
-    public void testXMLJS() {
-        LOG.info("Javascript: native XML parser");
-
-        controller.enqueue("<a><b foo='bar'>Bad</b><b good='true'>Good</b><b good='false'>Bad</b></a>".getBytes());
-        controller.enqueue("<a><b>Hello</b><b>world</b></a>".getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/parseXml.js");
-        controller.run(2);
-
-        assertRelationshipContents("Good", "success");
-        assertRelationshipContents("<a><b>Hello</b><b>world</b></a>", "failure");
-    }
-
-    @Test
-    public void testFlowFileJS() {
-        LOG.info("JavaScript: get FlowFile properties");
-
-        controller.enqueue("This is too short".getBytes());
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/ffTest.js");
-        controller.run(2);
-
-        assertRelationshipContents(multiline, "success");
-        assertRelationshipContents("This is too short", "failure");
-    }
-
-    @Test
-    public void testMultiThreadExecJS() {
-        LOG.info("JavaScript script 20 threads: Failing file based on reading contents");
-
-        Map<String, String> attrs1 = new HashMap<>();
-        attrs1.put("filename", "StuffIsFine.txt");
-        Map<String, String> attrs2 = new HashMap<>();
-        attrs2.put("filename", "multiline.txt");
-        controller.setThreadCount(20);
-        for (int i = 0; i < 10; i++) {
-            controller.enqueue("This stuff is fine".getBytes(), attrs1);
-            controller.enqueue(multiline.getBytes(), attrs2);
-        }
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.js");
-        controller.setProperty("expr", "rehenderit");
-        controller.run(20);
-
-        controller.assertTransferCount("failure", 10);
-        controller.assertTransferCount("success", 10);
-        for (int i = 0; i < 10; i++) {
-            MockFlowFile ff = controller.getFlowFilesForRelationship("failure").get(i);
-            ff.assertContentEquals(multiline);
-            assertTrue(ff.getAttribute("filename").endsWith("modified"));
-            ff = controller.getFlowFilesForRelationship("success").get(i);
-            ff.assertContentEquals("This stuff is fine");
-            assertTrue(ff.getAttribute("filename").endsWith("modified"));
-        }
-    }
-
-    @Test
-    public void testUpdateScriptJS() throws Exception {
-        LOG.info("Test one script with updated class");
-
-        File testFile = File.createTempFile("script", ".js");
-        File original = new File("target/test-scripts/readWithParams.js");
-        FileUtils.copyFile(original, testFile);
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, testFile.getPath());
-        controller.assertValid();
-        original = new File("target/test-scripts/optionalValidators.js");
-        FileUtils.copyFile(original, testFile);
-        controller.setProperty(ExecuteScript.SCRIPT_CHECK_INTERVAL, "5 secs");
-        Thread.sleep(6000);
-
-        controller.assertNotValid();
-        controller.setProperty("int", "abc");
-        controller.assertNotValid();
-        controller.setProperty("url", "not@valid");
-        controller.assertNotValid();
-        controller.setProperty("nonEmpty", "");
-        controller.assertNotValid();
-
-        controller.setProperty("int", "123");
-        controller.setProperty("url", "http://localhost");
-        controller.setProperty("nonEmpty", "abc123");
-        controller.assertValid();
-        FileUtils.deleteQuietly(testFile);
-    }
-
-    // ////////////////////////////////// // Python script tests
-    @Test
-    public void testSimpleReadP() {
-        LOG.info("Python script: fail file based on reading contents");
-
-        for (int i = 0; i < 20; i++) {
-            Map<String, String> attr1 = new HashMap<>();
-            attr1.put("filename", "FineStuff");
-            attr1.put("counter", Integer.toString(i));
-            Map<String, String> attr2 = new HashMap<>();
-            attr2.put("filename", "MultiLine");
-            attr2.put("counter", Integer.toString(i));
-            controller.enqueue("This stuff is fine".getBytes(), attr1);
-            controller.enqueue(multiline.getBytes(), attr2);
-        }
-
-        controller.setThreadCount(40);
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readTest.py");
-        controller.run(40);
-
-        assertRelationshipContents(multiline, "failure", 20);
-        assertRelationshipContents("This stuff is fine", "success", 20);
-
-        List<MockFlowFile> fails = controller.getFlowFilesForRelationship("failure");
-        List<MockFlowFile> successes = controller.getFlowFilesForRelationship("success");
-        for (int i = 0; i < 20; i++) {
-            assertTrue(fails.get(i).getAttribute("filename").matches("^.*\\d+$"));
-            assertTrue(successes.get(i).getAttribute("filename").matches("^.*\\d+$"));
-        }
-    }
-
-    @Test
-    public void testParamReadP() {
-        LOG.info("Python script: read contents and fail based on parameter");
-
-        controller.enqueue("This stuff is fine".getBytes());
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.py");
-        controller.setProperty("expr", "sed do");
-        controller.run(2);
-
-        assertRelationshipContents(multiline, "failure");
-        assertRelationshipContents("This stuff is fine", "success");
-    }
-
-    @Test
-    public void testWriteLastLineP() {
-        LOG.info("Running Python script to output last line of file");
-
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/writeTest.py");
-        controller.run();
-
-        List<MockFlowFile> sunkFiles = controller.getFlowFilesForRelationship("success");
-
-        assertEquals("Process did not generate an output file", 1, sunkFiles.size());
-
-        MockFlowFile sunkFile = sunkFiles.iterator().next();
-        byte[] blob = sunkFile.toByteArray();
-        String[] lines = new String(blob).split("\n");
-
-        assertEquals("File had more than one line", 1, lines.length);
-        assertEquals("sunt in culpa qui officia deserunt mollit anim id est laborum.", lines[0]);
-    }
-
-    @Test
-    public void testWriteOptionalParametersP() {
-        LOG.info("Python script processCallback that uses optional parameters");
-
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/paramTest.py");
-        controller.setProperty("repeat", "3");
-        controller.run();
-
-        List<MockFlowFile> sunkFiles = controller.getFlowFilesForRelationship("success");
-
-        assertEquals("Process did not generate an output file", 1, sunkFiles.size());
-
-        MockFlowFile sunkFile = sunkFiles.iterator().next();
-        byte[] blob = sunkFile.toByteArray();
-        String[] lines = new String(blob).split("\n");
-
-        assertEquals("File did not have 3 lines", 3, lines.length);
-        assertTrue(lines[2].startsWith("sunt in culpa qui officia deserunt mollit anim id est laborum."));
-    }
-
-    @Test
-    public void testManualValidationP() {
-        LOG.info("Python defining manual validator");
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/alwaysFail.py");
-        controller.assertNotValid();
-    }
-
-    @Test
-    public void testSetupOptionalValidationP() {
-        LOG.info("Python script creating validators for optional properties");
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/optionalValidators.py");
-        controller.setProperty("int", "abc");
-        controller.setProperty("url", "not@valid");
-        controller.setProperty("nonEmpty", "");
-        assertEquals(2, controller.getProcessor().getPropertyDescriptors().size());
-        controller.assertNotValid();
-
-        controller.setProperty("int", "123");
-        controller.setProperty("url", "http://localhost");
-        controller.setProperty("nonEmpty", "abc123");
-        assertEquals(5, controller.getProcessor().getPropertyDescriptors().size());
-        controller.assertValid();
-    }
-
-    @Test
-    public void testGetExceptionRouteP() {
-        LOG.info("Python script defining route taken in event of exception");
-
-        controller.enqueue("This stuff is fine".getBytes());
-        controller.enqueue("Bad things go to 'b'.".getBytes());
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/routeTest.py");
-
-        // Don't put the error in the logs
-        // TestableAppender ta = new TestableAppender();
-        // ta.attach(Logger.getLogger(ExecuteScript.class));
-        controller.run(3);
-        // ta.detach();
-
-        assertRelationshipContents("This stuff is fine", "a");
-        assertRelationshipContents("Bad things go to 'b'.", "b");
-        assertRelationshipContents(multiline, "c");
-
-        // ta.assertFound("threw exception");
-    }
-
-    @Test
-    public void testLoadLocalP() throws Exception {
-
-        final Thread t = new Thread(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    testGetExceptionRouteP();
-                    setupEach();
-                } catch (Exception e) {
-
-                }
-            }
-        });
-
-        t.start();
-        t.join();
-
-        LOG.info("Python: load another script file");
-
-        controller.enqueue("This stuff is fine".getBytes());
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/loadLocal.py");
-        controller.run(2);
-
-        assertRelationshipContents(multiline, "failure");
-        assertRelationshipContents("This stuff is fine", "success");
-    }
-
-    @Test
-    public void testSimpleConverterP() {
-        LOG.info("Running Python converter script");
-
-        for (int i = 0; i < 20; i++) {
-            controller.enqueue(multiline.getBytes());
-        }
-
-        controller.setThreadCount(20);
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/simpleConverter.py");
-        controller.run(20);
-
-        List<MockFlowFile> successFiles = controller.getFlowFilesForRelationship("success");
-        List<MockFlowFile> failFiles = controller.getFlowFilesForRelationship("failure");
-
-        assertEquals("Process did not generate 20 SUCCESS files", 20, successFiles.size());
-        assertEquals("Process did not generate 20 FAILURE files", 20, failFiles.size());
-
-        MockFlowFile sFile = successFiles.iterator().next();
-        MockFlowFile fFile = failFiles.iterator().next();
-
-        byte[] blob = sFile.toByteArray();
-        String[] lines = new String(blob).split("\n");
-
-        assertEquals("SUCCESS had wrong number of lines", 7, lines.length);
-        assertTrue(lines[0].startsWith("consectetur adipisicing elit,"));
-
-        blob = fFile.toByteArray();
-        lines = new String(blob).split("\n");
-
-        assertEquals("File had more than one line", 1, lines.length);
-        assertTrue(lines[0].startsWith("Lorem ipsum dolor sit amet,"));
-    }
-
-    @Test
-    public void testFlowFileP() {
-        LOG.info("Python: get FlowFile properties");
-
-        controller.enqueue("This is too short".getBytes());
-        controller.enqueue(multiline.getBytes());
-
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/ffTest.py");
-        controller.run(2);
-
-        assertRelationshipContents(multiline, "success");
-        assertRelationshipContents("This is too short", "failure");
-    }
-
-    @Test
-    public void testMultiThreadExecP() {
-        LOG.info("Pthon script 20 threads: Failing file based on reading contents");
-
-        Map<String, String> attrs1 = new HashMap<>();
-        attrs1.put("filename", "StuffIsFine.txt");
-        Map<String, String> attrs2 = new HashMap<>();
-        attrs2.put("filename", "multiline.txt");
-        for (int i = 0; i < 10; i++) {
-            controller.enqueue("This stuff is fine".getBytes(), attrs1);
-            controller.enqueue(multiline.getBytes(), attrs2);
-        }
-
-        controller.setThreadCount(20);
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.py");
-        controller.setProperty("expr", "sed do");
-        controller.run(20);
-
-        controller.assertTransferCount("failure", 10);
-        controller.assertTransferCount("success", 10);
-        for (int i = 0; i < 10; i++) {
-            MockFlowFile ff = controller.getFlowFilesForRelationship("failure").get(i);
-            ff.assertContentEquals(multiline);
-            assertTrue(ff.getAttribute("filename").endsWith("modified"));
-            ff = controller.getFlowFilesForRelationship("success").get(i);
-            ff.assertContentEquals("This stuff is fine");
-            assertTrue(ff.getAttribute("filename").endsWith("modified"));
-        }
-    }
-
-    @Test
-    public void testUpdateScriptP() throws Exception {
-        LOG.info("Test one script with updated class");
-
-        File testFile = File.createTempFile("script", ".py");
-        File original = new File("target/test-scripts/readTest.py");
-        FileUtils.copyFile(original, testFile);
-        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, testFile.getPath());
-        controller.assertValid();
-        original = new File("target/test-scripts/readWithParams.py");
-        FileUtils.copyFile(original, testFile);
-        controller.setProperty(ExecuteScript.SCRIPT_CHECK_INTERVAL, "5 secs");
-        Thread.sleep(6000);
-
-        controller.assertNotValid(); // need to set 'expr'
-        controller.setProperty("int", "abc");
-        controller.assertNotValid();
-        controller.setProperty("url", "not@valid");
-        controller.assertNotValid();
-        controller.setProperty("nonEmpty", "");
-        controller.assertNotValid();
-
-        controller.setProperty("expr", "sed do");
-        controller.assertValid();
-        assertEquals(6, controller.getProcessContext().getProperties().size());
-        FileUtils.deleteQuietly(testFile);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.js
deleted file mode 100644
index 40f038d..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.js
+++ /dev/null
@@ -1,24 +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.
- */
-with (Scripting) {
-    var instance = new ReaderScript({
-        validate: function () {
-            return ["This will never work."];
-        }
-    });
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.py
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.py
deleted file mode 100644
index 4b744ab..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.py
+++ /dev/null
@@ -1,19 +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.
-# type(name, superclass_tuple, member_dict) is a shorthand for defining an
-# anonymous class.  Note the trailing parens (), because scriptBuilder must
-# return an *instance* of the class.
-instance = type("FailingReader", (ReaderScript, object),\
-        {"validate": lambda self : ["I am broken"]})()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.rb
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.rb
deleted file mode 100644
index 076cdfa..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.rb
+++ /dev/null
@@ -1,21 +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.
-class NeverSatisfied < ReaderScript
-  def validate
-    return ["This is supposed to fail"]
-  end
-end
-
-NeverSatisfied.new

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.js
deleted file mode 100644
index d40623e..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.js
+++ /dev/null
@@ -1,28 +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.
- */
-with (Scripting) {
-    var instance = new ReaderScript({
-        route: function (input) {
-            if (instance.getFlowFileSize() < 20) {
-                return Script.FAIL_RELATIONSHIP;
-            } else {
-                return Script.SUCCESS_RELATIONSHIP;
-            }
-        }
-    });
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.py
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.py
deleted file mode 100644
index f4deb22..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.py
+++ /dev/null
@@ -1,22 +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.
-class SimpleJythonReader(ReaderScript):
-    def route(self, input):
-        if self.getFlowFileSize() < 20 : return self.FAIL_RELATIONSHIP
-             
-        return self.SUCCESS_RELATIONSHIP
-
-
-instance = SimpleJythonReader()

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.rb
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.rb
deleted file mode 100644
index 0435ea7..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.rb
+++ /dev/null
@@ -1,30 +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.
-class SimpleJRubyReader < ReaderScript
-  field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP
-  
-  def getRelationships 
-    @@evict = Relationship::Builder.new.name("evict").description("some evicted stuff").build()
-    [FAIL_RELATIONSHIP, SUCCESS_RELATIONSHIP, @@evict]
-  end
-  
-  def route( input )
-    return FAIL_RELATIONSHIP if getFlowFileSize < 20
-	return @@evict if !getAttribute("evict").nil?
-	return SUCCESS_RELATIONSHIP
-	end
-end
-
-SimpleJRubyReader.new

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/Sub.py
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/Sub.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/Sub.py
deleted file mode 100644
index 827816c..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/Sub.py
+++ /dev/null
@@ -1,18 +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.
-class TestFilter:
-    def notAllowed(self):
-        return "^sed"
- 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.js
deleted file mode 100644
index 489036a..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.js
+++ /dev/null
@@ -1,22 +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.
- */
-function notAllowed() {  // Works for eval(readFile(...))
-    return /sed do/i;
-}
-
-exports.notAllowed = notAllowed;  // Works for require(...)
-

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.rb
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.rb
deleted file mode 100644
index 775aad4..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.rb
+++ /dev/null
@@ -1,17 +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.
-def notAllowed
-  return /^sed/i
-end
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.js
deleted file mode 100644
index 39396ab..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.js
+++ /dev/null
@@ -1,30 +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.
- */
-var sub = require("lib/sub.js");
-
-with (Scripting) {
-    var instance = new ReaderScript({
-        route: function (input) {
-            var str = IOUtils.toString(input);
-            if (str.match(sub.notAllowed())) {
-                return Script.FAIL_RELATIONSHIP;
-            } else {
-                return Script.SUCCESS_RELATIONSHIP;
-            }
-        }
-    });
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.py
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.py
deleted file mode 100644
index 2429ea2..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.py
+++ /dev/null
@@ -1,26 +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.
-import re
-from Sub import TestFilter
-class SimpleJythonReader(ReaderScript):
-    def route(self, input):
-        tf = TestFilter()
-        for line in FileUtil.wrap(input):
-            if re.match(tf.notAllowed(),line): return self.FAIL_RELATIONSHIP
-             
-        return self.SUCCESS_RELATIONSHIP
-
-
-instance = SimpleJythonReader()

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.rb
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.rb
deleted file mode 100644
index f1b3ec5..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.rb
+++ /dev/null
@@ -1,29 +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.
-require 'sub'
-
-class SimpleJRubyReader < ReaderScript
-  field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP
-  
-  def route( input )
-    input.to_io.each_line do |line|
-      return FAIL_RELATIONSHIP if line.match notAllowed
-    end
-
-    return SUCCESS_RELATIONSHIP
-  end
-end
-
-SimpleJRubyReader.new

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/log4j.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/log4j.xml b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/log4j.xml
deleted file mode 100644
index d984dff..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/log4j.xml
+++ /dev/null
@@ -1,54 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" ?>
-<!--
-  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.
--->
-<!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
-
-<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
-    <appender name="CONSOLE" class="org.apache.log4j.ConsoleAppender">
-        <param name="Target" value="System.out"/>
-        <param name="Threshold" value="DEBUG"/>
-
-        <layout class="org.apache.log4j.PatternLayout">
-            <param name="ConversionPattern" value="%d{MM-dd-yy HH:mm:ss.SSS} %-5p %c{2} %x - %m%n"/>
-        </layout>
-
-    </appender>
-
-    <logger name="org.nifi.model.processor.FlowFileProcessor" additivity="false">
-        <level value="INFO"/>
-        <appender-ref ref="CONSOLE"/>
-    </logger>
-
-    <logger name="junit.TestableAppender" additivity="false">
-        <level value="TRACE"/>
-        <appender-ref ref="CONSOLE"/>
-    </logger>
-
-    <logger name="org.nifi.model.processor.impl.ScriptRunnerProcessor" additivity="false">
-        <level value="WARN"/>
-        <appender-ref ref="CONSOLE"/>
-    </logger>
-  
-    <logger name="org.nifi.model.processor.impl.ScriptRunnerProcessorTest" additivity="false">
-        <level value="INFO"/>
-        <appender-ref ref="CONSOLE"/>
-    </logger>
-
-    <root>
-        <level value="INFO"/>
-        <appender-ref ref="CONSOLE" />
-    </root>
-
-</log4j:configuration>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.js
deleted file mode 100644
index cf7c551..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.js
+++ /dev/null
@@ -1,28 +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.
- */
-with (Scripting) {
-    var instance = new WriterScript({
-        getPropertyDescriptors: function () {
-            i = new PropertyDescriptor.Builder().name("int").description("an int").required(true).addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR).build();
-            u = new PropertyDescriptor.Builder().name("url").description("a url").required(true).addValidator(StandardValidators.URL_VALIDATOR).build();
-            s = new PropertyDescriptor.Builder().name("nonEmpty").description("a non empty property").required(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
-            return [i, u, s];
-        }
-    });
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.py
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.py
deleted file mode 100644
index 9f1a43d..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.py
+++ /dev/null
@@ -1,22 +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.
-class SimpleJythonReader(ReaderScript):
-    def getPropertyDescriptors( self ):
-        nev = PropertyDescriptor.Builder().name("nonEmpty").required(1).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build()
-        iv = PropertyDescriptor.Builder().name("int").required(1).addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR).build()
-        uv = PropertyDescriptor.Builder().name("url").required(1).addValidator(StandardValidators.URL_VALIDATOR).build()
-        return [nev, iv, uv]
-
-instance = SimpleJythonReader()

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.rb
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.rb
deleted file mode 100644
index 3fa53b4..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.rb
+++ /dev/null
@@ -1,39 +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.
-class SimpleJRubyReader < ReaderScript
-  field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP, :logger
-  
-  def getPropertyDescriptors
-    logger.debug("Defining descriptors");
-    i = StandardValidators::INTEGER_VALIDATOR
-    u = StandardValidators::URL_VALIDATOR
-    s = StandardValidators::NON_EMPTY_VALIDATOR
-    intPropDesc = PropertyDescriptor::Builder.new().name("int").required(true).addValidator(i).build()
-    urlPropDesc = PropertyDescriptor::Builder.new().name("url").required(true).addValidator(u).build()
-    nonEmptyPropDesc = PropertyDescriptor::Builder.new().name("nonEmpty").addValidator(s).build()
-    return [intPropDesc, urlPropDesc, nonEmptyPropDesc]
-  end
-  
-  def route( input )
-    logger.debug("Routing input");
-    input.to_io.each_line do |line|
-      return FAIL_RELATIONSHIP if line.match /^sed/i
-    end
-
-    return SUCCESS_RELATIONSHIP
-	end
-end
-$logger.debug("Creating SimpleJRubyReader with props" + @properties.to_s)
-SimpleJRubyReader.new
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.js
deleted file mode 100644
index c00537a..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.js
+++ /dev/null
@@ -1,28 +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.
- */
-with (Scripting) {
-    var instance = new WriterScript({
-        process: function (input, output) {
-            var str = IOUtils.toString(input);
-            var last = str.split("\n").pop() + "\n";
-            for (var i = 0; i < instance.getProperty("repeat"); i++) {
-                IOUtils.write(last, output);
-            }
-            output.flush();
-        }
-    });
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.py
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.py
deleted file mode 100644
index 3495e4a..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.py
+++ /dev/null
@@ -1,26 +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.
-class ParamUsingWriter(WriterScript):
-    def process ( self, input, output ):
-        last = FileUtil.wrap(input).readlines()[-1] + '\n'
-        writer = FileUtil.wrap(output)
-        times = int(self.getProperty("repeat"))
-        lines = [last] * times
-
-        writer.writelines(lines)
-
-        writer.close()
-
-instance = ParamUsingWriter()

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.rb
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.rb
deleted file mode 100644
index 7c2eb23..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.rb
+++ /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.
-class ParamUsingWriter < WriterScript
-  def process ( input, output )
-    reader = input.to_io
-    writer = output.to_io
-    
-    last = reader.readlines.last
-    
-    getProperty("repeat").to_i.times do
-      writer << last + "\n"
-    end
-    
-    writer.close
-    reader.close
-  end
-end
-
-ParamUsingWriter.new

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/parseXml.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/parseXml.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/parseXml.js
deleted file mode 100644
index 901e158..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/parseXml.js
+++ /dev/null
@@ -1,36 +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.
- */
-with (Scripting) {
-    var instance = new WriterScript({
-        process: function (istream, ostream) {
-            var str = IOUtils.toString(istream);
-            var obj = new XML(str);
-            print(obj)
-            var good = obj.b.(@good == "true");
-                    if (good.length() == 0) {
-                instance.setRoute(Script.FAIL_RELATIONSHIP);
-                IOUtils.write(str, ostream);
-            } else {
-                instance.setRoute(Script.SUCCESS_RELATIONSHIP);
-                for each (var goodStr in good) {
-                    IOUtils.write(goodStr, ostream);
-                }
-            }
-            ostream.flush();
-        }
-    });
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.js
deleted file mode 100644
index f5d3ecc..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.js
+++ /dev/null
@@ -1,30 +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.
- */
-with (Scripting) {
-
-    var instance = new ReaderScript({
-        route: function (input) {
-            str = IOUtils.toString(input);
-            if (str.match(/sed do/i)) {
-                return Script.FAIL_RELATIONSHIP;
-            } else {
-                return Script.SUCCESS_RELATIONSHIP;
-            }
-        }
-    });
-    logger.debug("Got a logger and properties" + properties);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.py
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.py
deleted file mode 100644
index 91e6ca7..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.py
+++ /dev/null
@@ -1,32 +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.
-import os, re, subprocess
-
-class SimpleJythonReader(ReaderScript):
-    def route(self, input):
-        logger.info("In route")
-        returnid = os.system("c:\\cygwin\\bin\\echo GOOD")
-        fname = self.getAttribute("filename")
-        counter = self.getAttribute("counter")
-        fname = fname + '.' + counter
-        self.setAttribute("filename", fname)
-        for line in FileUtil.wrap(input):
-            if re.match("^sed",line): return self.FAIL_RELATIONSHIP
-
-        return self.SUCCESS_RELATIONSHIP
-
-
-
-instance = SimpleJythonReader()

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.rb
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.rb
deleted file mode 100644
index c10765d..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.rb
+++ /dev/null
@@ -1,30 +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.
-class SimpleJRubyReader < ReaderScript
-  field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP, :logger, :attributes
-
-  def route( input )
-    logger.info("Route Input")
-    input.to_io.each_line do |line|
-      return FAIL_RELATIONSHIP if line.match /^sed/i
-    end
-
-	attributes.put("filename", "NewFileNameFromReadTest")
-    return SUCCESS_RELATIONSHIP
-  end
-end
-$logger.info("Logger is made available in shared variables...however, the SimpleJRubyReader.logger is not set till after this script returns")
-
-SimpleJRubyReader.new

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.js
deleted file mode 100644
index e34ad5f..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.js
+++ /dev/null
@@ -1,32 +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.
- */
-with (Scripting) {
-    var instance = new ReaderScript({
-        route: function (input) {
-            var str = IOUtils.toString(input);
-            var expr = instance.getProperty("expr");
-            filename = instance.attributes.get("filename");
-            instance.setAttribute("filename", filename + ".modified");
-            if (str.match(expr)) {
-                return Script.FAIL_RELATIONSHIP;
-            } else {
-                return Script.SUCCESS_RELATIONSHIP;
-            }
-        }
-    });
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.py
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.py
deleted file mode 100644
index 23e55af..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.py
+++ /dev/null
@@ -1,32 +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.
-import re
-
-class SimpleJythonReader(ReaderScript):
-    def getPropertyDescriptors( self ):
-        nev = StandardValidators.NON_EMPTY_VALIDATOR   
-        return [PropertyDescriptor.Builder().name("expr").required(1).addValidator(nev).build()]
-        
-    def route( self, input ):
-        expr = self.getProperty("expr")
-        filename = self.getAttribute("filename")
-        self.setAttribute("filename", filename + ".modified")
-        for line in FileUtil.wrap(input):
-            if re.match(expr, line): return self.FAIL_RELATIONSHIP 
-
-        return self.SUCCESS_RELATIONSHIP
-
-instance = SimpleJythonReader()
-

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.rb
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.rb
deleted file mode 100644
index 308b652..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.rb
+++ /dev/null
@@ -1,33 +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.
-class SimpleJRubyReader < ReaderScript
-  field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP, :properties, :attributes
-  
-  def route( input )
-    expr = properties.get "expr"
-    raise "Must specify the 'expr' property!" if expr.nil?
-	filename = attributes.get "filename"
-	setAttribute("filename", filename + ".modified")
-    input.to_io.each_line do |line|
-      return FAIL_RELATIONSHIP if line.match expr
-    end
-
-    return SUCCESS_RELATIONSHIP
-  end
-end
-
-$logger.debug("Can access logger and properties via shared instance variables...props = " + @properties.to_s)
-SimpleJRubyReader.new
-

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.js
deleted file mode 100644
index 47ef546..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.js
+++ /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.
- */
-with (Scripting) {
-    var a = new Relationship.Builder().name("a").description("some good stuff").build()
-    var b = new Relationship.Builder().name("b").description("some other stuff").build()
-    var c = new Relationship.Builder().name("c").description("some bad stuff").build()
-    var instance = new ReaderScript({
-        getExceptionRoute: function () {
-            return c;
-        },
-        getRelationships: function () {
-            return [a, b, c];
-        },
-        route: function (input) {
-            var str = IOUtils.toString(input);
-            var lines = str.split("\n");
-            for (var line in lines) {
-                if (lines[line].match(/^bad/i)) {
-                    return b;
-                } else if (lines[line].match(/^sed/i)) {
-                    throw "That's no good!";
-                }
-            }
-            return a;
-        }
-    });
-}


[50/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/assembly/pom.xml
----------------------------------------------------------------------
diff --git a/assembly/pom.xml b/assembly/pom.xml
deleted file mode 100644
index e785c45..0000000
--- a/assembly/pom.xml
+++ /dev/null
@@ -1,458 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-    <artifactId>nifi</artifactId>
-    <version>0.0.1-SNAPSHOT</version>
-    <packaging>pom</packaging>
-    <name>NiFi Release</name>
-    <description>This is the assembly Apache NiFi (incubating)</description>
-    <build>
-        <plugins>
-            <plugin>
-                <artifactId>maven-assembly-plugin</artifactId>
-                <configuration>
-                    <attach>true</attach>
-                </configuration>
-                <executions>
-                    <execution>
-                        <id>make shared resource</id>
-                        <goals>
-                            <goal>single</goal>
-                        </goals>
-                        <phase>package</phase>
-                        <configuration>
-                            <descriptors>
-                                <descriptor>src/main/assembly/dependencies.xml</descriptor>
-                            </descriptors>
-                        </configuration>
-                    </execution>
-                </executions>    
-            </plugin>
-        </plugins>
-    </build>
-    <dependencies>
-        <dependency>
-            <groupId>ch.qos.logback</groupId>
-            <artifactId>logback-classic</artifactId>
-            <scope>compile</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>jcl-over-slf4j</artifactId>
-            <scope>compile</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>jul-to-slf4j</artifactId>
-            <scope>compile</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>log4j-over-slf4j</artifactId>
-            <scope>compile</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>slf4j-api</artifactId>
-            <scope>compile</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-runtime</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-bootstrap</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-resources</artifactId>
-            <classifier>resources</classifier>
-            <scope>runtime</scope>
-            <type>zip</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-docs</artifactId>
-            <classifier>resources</classifier>
-            <scope>runtime</scope>
-            <type>zip</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-framework-nar</artifactId>
-            <type>nar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>volatile-provenance-repository-nar</artifactId>
-            <type>nar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>persistent-provenance-repository-nar</artifactId>
-            <type>nar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>standard-services-api-nar</artifactId>
-            <type>nar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>ssl-context-service-nar</artifactId>
-            <type>nar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>distributed-cache-services-nar</artifactId>
-            <type>nar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-standard-nar</artifactId>
-            <type>nar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-jetty-bundle</artifactId>
-            <type>nar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>update-attribute-nar</artifactId>
-            <type>nar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>monitor-threshold-nar</artifactId>
-            <type>nar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>hadoop-libraries-nar</artifactId>
-            <type>nar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>hadoop-nar</artifactId>
-            <type>nar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>kafka-nar</artifactId>
-            <type>nar</type>
-        </dependency>
-    </dependencies>
-    
-    <properties>        
-        <!--Wrapper Properties-->
-        <nifi.wrapper.jvm.heap.initial.mb>256</nifi.wrapper.jvm.heap.initial.mb>
-        <nifi.wrapper.jvm.heap.max.mb>512</nifi.wrapper.jvm.heap.max.mb>
-        <nifi.initial.permgen.size.mb>128</nifi.initial.permgen.size.mb>
-        <nifi.max.permgen.size.mb>128</nifi.max.permgen.size.mb>
-        <nifi.wrapper.logfile.maxsize>10m</nifi.wrapper.logfile.maxsize>
-        <nifi.wrapper.logfile.maxfiles>10</nifi.wrapper.logfile.maxfiles>
-        
-        <!-- nifi.properties: core properties -->
-        <nifi.flowcontroller.autoResumeState>true</nifi.flowcontroller.autoResumeState>
-        <nifi.flowcontroller.graceful.shutdown.period>10 sec</nifi.flowcontroller.graceful.shutdown.period>
-        <nifi.flowservice.writedelay.interval>500 ms</nifi.flowservice.writedelay.interval>
-        <nifi.administrative.yield.duration>30 sec</nifi.administrative.yield.duration>
-
-        <nifi.flow.configuration.file>./conf/flow.xml.gz</nifi.flow.configuration.file>
-        <nifi.flow.configuration.archive.dir>./conf/archive/</nifi.flow.configuration.archive.dir>
-        <nifi.reporting.task.configuration.file>./conf/reporting-tasks.xml</nifi.reporting.task.configuration.file>
-        <nifi.controller.service.configuration.file>./conf/controller-services.xml</nifi.controller.service.configuration.file>
-        <nifi.authority.provider.configuration.file>./conf/authority-providers.xml</nifi.authority.provider.configuration.file>
-        <nifi.templates.directory>./conf/templates</nifi.templates.directory>
-        <nifi.database.directory>./database_repository</nifi.database.directory>
-
-        <nifi.flowfile.repository.implementation>org.apache.nifi.controller.repository.WriteAheadFlowFileRepository</nifi.flowfile.repository.implementation>
-        <nifi.flowfile.repository.directory>./flowfile_repository</nifi.flowfile.repository.directory>
-        <nifi.flowfile.repository.partitions>256</nifi.flowfile.repository.partitions>
-        <nifi.flowfile.repository.checkpoint.interval>2 mins</nifi.flowfile.repository.checkpoint.interval>
-        <nifi.flowfile.repository.always.sync>false</nifi.flowfile.repository.always.sync>
-        <nifi.swap.manager.implementation>org.apache.nifi.controller.FileSystemSwapManager</nifi.swap.manager.implementation>
-        <nifi.queue.swap.threshold>20000</nifi.queue.swap.threshold>
-        <nifi.swap.in.period>5 sec</nifi.swap.in.period>
-        <nifi.swap.in.threads>1</nifi.swap.in.threads>
-        <nifi.swap.out.period>5 sec</nifi.swap.out.period>
-        <nifi.swap.out.threads>4</nifi.swap.out.threads>
-		
-        <nifi.content.repository.implementation>org.apache.nifi.controller.repository.FileSystemRepository</nifi.content.repository.implementation>
-        <nifi.content.claim.max.appendable.size>10 MB</nifi.content.claim.max.appendable.size>
-        <nifi.content.claim.max.flow.files>100</nifi.content.claim.max.flow.files>
-        <nifi.content.repository.directory.default>./content_repository</nifi.content.repository.directory.default>
-        <nifi.content.repository.archive.max.retention.period />
-        <nifi.content.repository.archive.max.usage.percentage />
-        <nifi.content.repository.archive.enabled>false</nifi.content.repository.archive.enabled>
-        <nifi.content.repository.always.sync>false</nifi.content.repository.always.sync>
-        <nifi.content.viewer.url />
-        
-        
-        <nifi.restore.directory />
-        <nifi.ui.banner.text></nifi.ui.banner.text>
-        <nifi.ui.autorefresh.interval>30 sec</nifi.ui.autorefresh.interval>
-        <nifi.nar.library.directory>./lib</nifi.nar.library.directory>
-        <nifi.nar.working.directory>./work/nar/</nifi.nar.working.directory>
-        <nifi.documentation.working.directory>./work/docs/components</nifi.documentation.working.directory>
-        
-        <nifi.sensitive.props.algorithm>PBEWITHMD5AND256BITAES-CBC-OPENSSL</nifi.sensitive.props.algorithm>
-        <nifi.sensitive.props.provider>BC</nifi.sensitive.props.provider>
-        <nifi.h2.url.append>;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE</nifi.h2.url.append>
-
-        <nifi.remote.input.socket.port>9990</nifi.remote.input.socket.port>
-        
-        <!-- persistent provenance repository properties -->
-        <nifi.provenance.repository.implementation>org.apache.nifi.provenance.PersistentProvenanceRepository</nifi.provenance.repository.implementation>
-        <nifi.provenance.repository.directory.default>./provenance_repository</nifi.provenance.repository.directory.default>
-        <nifi.provenance.repository.max.storage.time>24 hours</nifi.provenance.repository.max.storage.time>
-        <nifi.provenance.repository.max.storage.size>1 GB</nifi.provenance.repository.max.storage.size>
-        <nifi.provenance.repository.rollover.time>5 mins</nifi.provenance.repository.rollover.time>
-        <nifi.provenance.repository.rollover.size>100 MB</nifi.provenance.repository.rollover.size>
-        <nifi.provenance.repository.query.threads>2</nifi.provenance.repository.query.threads>
-        <nifi.provenance.repository.compress.on.rollover>true</nifi.provenance.repository.compress.on.rollover>
-        <nifi.provenance.repository.indexed.fields>EventType, FlowFileUUID, Filename, ProcessorID</nifi.provenance.repository.indexed.fields>
-        <nifi.provenance.repository.indexed.attributes />
-        <nifi.provenance.repository.index.shard.size>500 MB</nifi.provenance.repository.index.shard.size>
-        <nifi.provenance.repository.always.sync>false</nifi.provenance.repository.always.sync>
-        <nifi.provenance.repository.journal.count>16</nifi.provenance.repository.journal.count>
-        
-        <!-- volatile provenance repository properties -->
-        <nifi.provenance.repository.buffer.size>100000</nifi.provenance.repository.buffer.size>
-        
-        <!-- Component status repository properties -->
-        <nifi.components.status.repository.implementation>org.apache.nifi.controller.status.history.VolatileComponentStatusRepository</nifi.components.status.repository.implementation>
-        <nifi.components.status.repository.buffer.size>288</nifi.components.status.repository.buffer.size>
-        <nifi.components.status.snapshot.frequency>5 mins</nifi.components.status.snapshot.frequency>
-        
-        <!-- nifi.properties: web properties -->
-        <nifi.web.war.directory>./lib</nifi.web.war.directory>
-        <nifi.web.http.host />
-        <nifi.web.http.port>8080</nifi.web.http.port>
-        <nifi.web.https.host />
-        <nifi.web.https.port />
-        <nifi.jetty.work.dir>./work/jetty</nifi.jetty.work.dir>
-        <nifi.web.jetty.threads>200</nifi.web.jetty.threads>
-        
-        <!-- nifi.properties: security properties -->
-        <nifi.security.keystore />
-        <nifi.security.keystoreType />
-        <nifi.security.keystorePasswd />
-        <nifi.security.keyPasswd />
-        <nifi.security.truststore />
-        <nifi.security.truststoreType />
-        <nifi.security.truststorePasswd />
-        <nifi.security.needClientAuth />
-        <nifi.security.authorizedUsers.file>./conf/authorized-users.xml</nifi.security.authorizedUsers.file>
-        <nifi.security.user.credential.cache.duration>24 hours</nifi.security.user.credential.cache.duration>
-        <nifi.security.user.authority.provider>file-provider</nifi.security.user.authority.provider>
-        <nifi.security.x509.principal.extractor />
-        <nifi.security.support.new.account.requests />
-        <nifi.security.ocsp.responder.url />
-        <nifi.security.ocsp.responder.certificate />
-        
-        <!-- nifi.properties: cluster common properties (cluster manager and nodes must have same values) -->
-        <nifi.cluster.protocol.heartbeat.interval>5 sec</nifi.cluster.protocol.heartbeat.interval>
-        <nifi.cluster.protocol.is.secure>false</nifi.cluster.protocol.is.secure>
-        <nifi.cluster.protocol.socket.timeout>30 sec</nifi.cluster.protocol.socket.timeout>
-        <nifi.cluster.protocol.connection.handshake.timeout>45 sec</nifi.cluster.protocol.connection.handshake.timeout> 
-        <nifi.cluster.protocol.use.multicast>false</nifi.cluster.protocol.use.multicast>
-        <nifi.cluster.protocol.multicast.address />
-        <nifi.cluster.protocol.multicast.port />
-        <nifi.cluster.protocol.multicast.service.broadcast.delay>500 ms</nifi.cluster.protocol.multicast.service.broadcast.delay>
-        <nifi.cluster.protocol.multicast.service.locator.attempts>3</nifi.cluster.protocol.multicast.service.locator.attempts>
-        <nifi.cluster.protocol.multicast.service.locator.attempts.delay>1 sec</nifi.cluster.protocol.multicast.service.locator.attempts.delay>
-
-        <!-- nifi.properties: cluster node properties (only configure for cluster nodes) -->
-        <nifi.cluster.is.node>false</nifi.cluster.is.node>
-        <nifi.cluster.node.address />
-        <nifi.cluster.node.protocol.port />
-        <nifi.cluster.node.protocol.threads>2</nifi.cluster.node.protocol.threads>
-        <nifi.cluster.node.unicast.manager.address />
-        <nifi.cluster.node.unicast.manager.protocol.port />
-        
-        <!-- nifi.properties: cluster manager properties (only configure for cluster manager) -->
-        <nifi.cluster.is.manager>false</nifi.cluster.is.manager>
-        <nifi.cluster.manager.address />
-        <nifi.cluster.manager.protocol.port />
-        <nifi.cluster.manager.node.firewall.file />
-        <nifi.cluster.manager.node.event.history.size>10</nifi.cluster.manager.node.event.history.size>
-        <nifi.cluster.manager.node.api.connection.timeout>30 sec</nifi.cluster.manager.node.api.connection.timeout>
-        <nifi.cluster.manager.node.api.read.timeout>30 sec</nifi.cluster.manager.node.api.read.timeout>
-        <nifi.cluster.manager.node.api.request.threads>10</nifi.cluster.manager.node.api.request.threads>
-        <nifi.cluster.manager.flow.retrieval.delay>5 sec</nifi.cluster.manager.flow.retrieval.delay>
-        <nifi.cluster.manager.protocol.threads>10</nifi.cluster.manager.protocol.threads>
-        <nifi.cluster.manager.safemode.duration>0 sec</nifi.cluster.manager.safemode.duration>
-    </properties>
-    <profiles>
-        <profile>
-            <id>rpm</id>
-            <activation>
-                <activeByDefault>false</activeByDefault>
-            </activation>
-            <build>
-                <plugins>
-                    <plugin>
-                        <artifactId>maven-dependency-plugin</artifactId>
-                        <executions>
-                            <execution>
-                                <id>unpack-shared-resources</id>
-                                <goals>
-                                    <goal>unpack-dependencies</goal>
-                                </goals>
-                                <phase>generate-resources</phase>
-                                <configuration>
-                                    <outputDirectory>${project.build.directory}/generated-resources</outputDirectory>
-                                    <includeArtifactIds>nifi-resources</includeArtifactIds>
-                                    <includeGroupIds>org.apache.nifi</includeGroupIds>
-                                    <excludeTransitive>false</excludeTransitive>
-                                </configuration>
-                            </execution>
-                            <execution>
-                                <id>unpack-docs</id>
-                                <goals>
-                                    <goal>unpack-dependencies</goal>
-                                </goals>
-                                <phase>generate-resources</phase>
-                                <configuration>
-                                    <outputDirectory>${project.build.directory}/generated-docs</outputDirectory>
-                                    <includeArtifactIds>nifi-docs</includeArtifactIds>
-                                    <includeGroupIds>org.apache.nifi</includeGroupIds>
-                                    <excludeTransitive>false</excludeTransitive>
-                                </configuration>
-                            </execution>
-                        </executions>
-                    </plugin>                    
-                    <plugin>
-                        <groupId>org.codehaus.mojo</groupId>
-                        <artifactId>rpm-maven-plugin</artifactId>
-                        <configuration>
-                            <summary>Apache NiFi (incubating)</summary>
-                            <description>Apache Nifi (incubating) is dataflow system based on the Flow-Based Programming concepts.</description>
-                            <license>Apache License, Version 2.0 and others (see included LICENSE file)</license>
-                            <url>http://nifi.incubator.apache.org</url>
-                            <group>Utilities</group>
-                            <prefix>/opt/nifi</prefix>
-                            <defineStatements>
-                                <defineStatement>_use_internal_dependency_generator 0</defineStatement>
-                            </defineStatements>
-                            <defaultDirmode>750</defaultDirmode>
-                            <defaultFilemode>640</defaultFilemode>
-                            <defaultUsername>root</defaultUsername>
-                            <defaultGroupname>root</defaultGroupname>
-                        </configuration>
-                        <executions>
-                            <execution>
-                                <id>build-bin-rpm</id>
-                                <goals>
-                                    <goal>attached-rpm</goal>
-                                </goals>
-                                <configuration>
-                                    <classifier>bin</classifier>
-                                    <provides>
-                                        <provide>nifi</provide>
-                                    </provides>
-                                    <mappings>
-                                        <mapping>
-                                            <directory>/opt/nifi/nifi-${project.version}</directory>
-                                        </mapping>
-                                        <mapping>
-                                            <directory>/opt/nifi/nifi-${project.version}</directory>
-                                            <sources>
-                                                <source>
-                                                    <location>../LICENSE</location>
-                                                </source>
-                                                <source>
-                                                    <location>../NOTICE</location>
-                                                </source>
-                                                <source>
-                                                    <location>../DISCLAIMER</location>
-                                                </source>
-                                                <source>
-                                                    <location>../README.md</location>
-                                                    <destination>README</destination>
-                                                </source>
-                                            </sources>
-                                        </mapping>
-                                        <mapping>
-                                            <directory>/opt/nifi/nifi-${project.version}/bin</directory>
-                                            <filemode>750</filemode>
-                                            <sources>
-                                                <source>
-                                                    <location>${project.build.directory}/generated-resources/bin/nifi.sh</location>
-                                                    <destination>nifi.sh</destination>
-                                                    <filter>true</filter>
-                                                </source>
-                                            </sources>
-                                        </mapping>
-                                        <mapping>
-                                            <directory>/opt/nifi/nifi-${project.version}/conf</directory>
-                                            <configuration>true</configuration>
-                                            <sources>
-                                                <source>
-                                                    <location>${project.build.directory}/generated-resources/conf</location>
-                                                    <filter>true</filter>
-                                                </source>
-                                            </sources>
-                                        </mapping>
-                                        <mapping>
-                                            <directory>/opt/nifi/nifi-${project.version}/lib</directory>
-                                            <dependency>
-                                                <excludes>
-                                                    <exclude>org.apache.nifi:nifi-bootstrap</exclude>
-                                                    <exclude>org.apache.nifi:nifi-resources</exclude>
-                                                    <exclude>org.apache.nifi:nifi-docs</exclude>
-                                                </excludes>
-                                            </dependency>
-                                        </mapping>
-                                        <mapping>
-                                            <directory>/opt/nifi/nifi-${project.version}/lib/bootstrap</directory>
-                                            <dependency>
-                                                <includes>
-                                                    <include>org.apache.nifi:nifi-bootstrap</include>
-                                                </includes>
-                                            </dependency>
-                                        </mapping>
-                                        <mapping>
-                                            <directory>/opt/nifi/nifi-${project.version}/docs</directory>
-                                            <sources>
-                                                <source>
-                                                    <location>${project.build.directory}/generated-docs</location>
-                                                </source>
-                                            </sources>
-                                        </mapping>
-                                    </mappings>
-                                </configuration>
-                            </execution>
-                        </executions>
-                    </plugin>
-                </plugins>
-            </build>
-        </profile>
-    </profiles>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/assembly/src/main/assembly/dependencies.xml
----------------------------------------------------------------------
diff --git a/assembly/src/main/assembly/dependencies.xml b/assembly/src/main/assembly/dependencies.xml
deleted file mode 100644
index 3481b0a..0000000
--- a/assembly/src/main/assembly/dependencies.xml
+++ /dev/null
@@ -1,140 +0,0 @@
-<?xml version="1.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.
--->
-<assembly>
-    <id>bin</id>
-    <formats>
-        <format>dir</format>
-        <format>zip</format>
-        <format>tar.gz</format>
-    </formats>
-    <includeBaseDirectory>true</includeBaseDirectory>
-    <baseDirectory>${project.artifactId}-${project.version}</baseDirectory>
-
-    <dependencySets>
-        <!-- Write out all dependency artifacts to lib directory -->
-        <dependencySet>
-            <scope>runtime</scope>
-            <useProjectArtifact>false</useProjectArtifact>
-            <outputDirectory>lib</outputDirectory>
-            <directoryMode>0750</directoryMode>
-            <fileMode>0640</fileMode>
-            <useTransitiveFiltering>true</useTransitiveFiltering>
-            <excludes>
-            	<exclude>nifi-bootstrap</exclude>
-                <exclude>nifi-resources</exclude>
-                <exclude>nifi-docs</exclude>
-            </excludes>
-        </dependencySet>
-        
-        <!-- Write out the bootstrap lib component to its own dir -->
-        <dependencySet>
-            <scope>runtime</scope>
-            <useProjectArtifact>false</useProjectArtifact>
-            <outputDirectory>lib/bootstrap</outputDirectory>
-            <directoryMode>0750</directoryMode>
-            <fileMode>0640</fileMode>
-            <useTransitiveFiltering>true</useTransitiveFiltering>
-            <includes>
-            	<include>nifi-bootstrap</include>
-            </includes>
-        </dependencySet>
-        
-        <!-- Write out the conf directory contents -->
-        <dependencySet>
-            <scope>runtime</scope>
-            <useProjectArtifact>false</useProjectArtifact>
-            <outputDirectory>./</outputDirectory>
-            <directoryMode>0750</directoryMode>
-            <fileMode>0640</fileMode>
-            <useTransitiveFiltering>true</useTransitiveFiltering>
-            <includes>
-            	<include>nifi-resources</include>
-            </includes>
-            <unpack>true</unpack>
-            <unpackOptions>
-                <filtered>true</filtered>
-                <includes>
-                    <include>conf/*</include>
-                </includes>
-            </unpackOptions>
-        </dependencySet>
-
-        <!-- Write out the bin directory contents -->
-        <dependencySet>
-            <scope>runtime</scope>
-            <useProjectArtifact>false</useProjectArtifact>
-            <outputDirectory>./</outputDirectory>
-            <directoryMode>0750</directoryMode>
-            <fileMode>0750</fileMode>
-            <useTransitiveFiltering>true</useTransitiveFiltering>
-            <includes>
-            	<include>nifi-resources</include>
-            </includes>
-            <unpack>true</unpack>
-            <unpackOptions>
-                <filtered>true</filtered>
-                <includes>
-                    <include>bin/*</include>
-                </includes>
-            </unpackOptions>
-        </dependencySet>
-        
-        <!-- Writes out the docs directory contents -->
-        <dependencySet>
-            <scope>runtime</scope>
-            <useProjectArtifact>false</useProjectArtifact>
-            <outputDirectory>docs/</outputDirectory>
-            <useTransitiveFiltering>true</useTransitiveFiltering>
-            <includes>
-            	<include>nifi-docs</include>
-            </includes>
-            <unpack>true</unpack>
-            <unpackOptions>
-                <filtered>false</filtered>
-            </unpackOptions>
-        </dependencySet>                
-    </dependencySets>
-    <files>
-        <file>
-            <source>../README.md</source>
-            <outputDirectory>./</outputDirectory>
-            <destName>README</destName>
-            <fileMode>0644</fileMode>
-            <filtered>true</filtered>
-        </file>
-        <file>
-            <source>../DISCLAIMER</source>
-            <outputDirectory>./</outputDirectory>
-            <destName>DISCLAIMER</destName>
-            <fileMode>0644</fileMode>
-            <filtered>true</filtered>
-        </file>
-        <file>
-            <source>../LICENSE</source>
-            <outputDirectory>./</outputDirectory>
-            <destName>LICENSE</destName>
-            <fileMode>0644</fileMode>
-            <filtered>true</filtered>
-        </file>       
-        <file>
-            <source>../NOTICE</source>
-            <outputDirectory>./</outputDirectory>
-            <destName>NOTICE</destName>
-            <fileMode>0644</fileMode>
-            <filtered>true</filtered>
-        </file>
-    </files>
-</assembly>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/data-provenance-utils/.gitignore
----------------------------------------------------------------------
diff --git a/commons/data-provenance-utils/.gitignore b/commons/data-provenance-utils/.gitignore
deleted file mode 100755
index 19f2e00..0000000
--- a/commons/data-provenance-utils/.gitignore
+++ /dev/null
@@ -1,2 +0,0 @@
-/target
-/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/data-provenance-utils/pom.xml
----------------------------------------------------------------------
diff --git a/commons/data-provenance-utils/pom.xml b/commons/data-provenance-utils/pom.xml
deleted file mode 100644
index 0024b70..0000000
--- a/commons/data-provenance-utils/pom.xml
+++ /dev/null
@@ -1,40 +0,0 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <!--
-      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.
-    -->
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-commons-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-
-    <artifactId>data-provenance-utils</artifactId>
-    <version>0.0.1-SNAPSHOT</version>
-    <packaging>jar</packaging>
-
-    <name>data-provenance-utils</name>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-utils</artifactId>
-        </dependency>
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/AsyncLineageSubmission.java
----------------------------------------------------------------------
diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/AsyncLineageSubmission.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/AsyncLineageSubmission.java
deleted file mode 100644
index dc24a93..0000000
--- a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/AsyncLineageSubmission.java
+++ /dev/null
@@ -1,87 +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.provenance;
-
-import java.util.Collection;
-import java.util.Date;
-import java.util.UUID;
-
-import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
-import org.apache.nifi.provenance.lineage.LineageComputationType;
-
-/**
- *
- */
-public class AsyncLineageSubmission implements ComputeLineageSubmission {
-    private final String lineageIdentifier = UUID.randomUUID().toString();
-    private final Date submissionTime = new Date();
-
-    private final LineageComputationType computationType;
-    private final Long eventId;
-    private final Collection<String> lineageFlowFileUuids;
-
-    private volatile boolean canceled = false;
-
-    private final StandardLineageResult result;
-
-    public AsyncLineageSubmission(final LineageComputationType computationType, final Long eventId, final Collection<String> lineageFlowFileUuids, final int numSteps) {
-        this.computationType = computationType;
-        this.eventId = eventId;
-        this.lineageFlowFileUuids = lineageFlowFileUuids;
-        this.result = new StandardLineageResult(numSteps, lineageFlowFileUuids);
-    }
-
-    @Override
-    public StandardLineageResult getResult() {
-        return result;
-    }
-
-    @Override
-    public Date getSubmissionTime() {
-        return submissionTime;
-    }
-
-    @Override
-    public String getLineageIdentifier() {
-        return lineageIdentifier;
-    }
-
-    @Override
-    public void cancel() {
-        this.canceled = true;
-    }
-
-    @Override
-    public boolean isCanceled() {
-        return canceled;
-    }
-
-    @Override
-    public LineageComputationType getLineageComputationType() {
-        return computationType;
-    }
-
-    @Override
-    public Long getExpandedEventId() {
-        return eventId;
-    }
-
-    @Override
-    public Collection<String> getLineageFlowFileUuids() {
-        return lineageFlowFileUuids;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/AsyncQuerySubmission.java
----------------------------------------------------------------------
diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/AsyncQuerySubmission.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/AsyncQuerySubmission.java
deleted file mode 100644
index 4244476..0000000
--- a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/AsyncQuerySubmission.java
+++ /dev/null
@@ -1,81 +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.provenance;
-
-import java.util.Date;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.provenance.search.Query;
-import org.apache.nifi.provenance.search.QuerySubmission;
-
-/**
- *
- */
-public class AsyncQuerySubmission implements QuerySubmission {
-
-    public static final int TTL = (int) TimeUnit.MILLISECONDS.convert(60, TimeUnit.SECONDS);
-
-    private final Date submissionTime = new Date();
-    private final Query query;
-
-    private volatile boolean canceled = false;
-    private final StandardQueryResult queryResult;
-
-    /**
-     * Constructs an AsyncQuerySubmission with the given query and the given
-     * number of steps, indicating how many results must be added to this
-     * AsyncQuerySubmission before it is considered finished
-     *
-     * @param query
-     * @param numSteps
-     */
-    public AsyncQuerySubmission(final Query query, final int numSteps) {
-        this.query = query;
-        queryResult = new StandardQueryResult(query, numSteps);
-    }
-
-    @Override
-    public Date getSubmissionTime() {
-        return submissionTime;
-    }
-
-    @Override
-    public String getQueryIdentifier() {
-        return query.getIdentifier();
-    }
-
-    @Override
-    public void cancel() {
-        this.canceled = true;
-        queryResult.cancel();
-    }
-
-    @Override
-    public boolean isCanceled() {
-        return canceled;
-    }
-
-    @Override
-    public Query getQuery() {
-        return query;
-    }
-
-    @Override
-    public StandardQueryResult getResult() {
-        return queryResult;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/NamedSearchableField.java
----------------------------------------------------------------------
diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/NamedSearchableField.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/NamedSearchableField.java
deleted file mode 100644
index dc2903f..0000000
--- a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/NamedSearchableField.java
+++ /dev/null
@@ -1,95 +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.provenance;
-
-import org.apache.nifi.provenance.search.SearchableField;
-import org.apache.nifi.provenance.search.SearchableFieldType;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- *
- */
-public class NamedSearchableField implements SearchableField {
-
-    private final String identifier;
-    private final String searchableName;
-    private final SearchableFieldType fieldType;
-    private final String friendlyName;
-    private final boolean attribute;
-
-    NamedSearchableField(final String identifier, final String searchableName, final String friendlyName, final boolean attribute) {
-        this(identifier, searchableName, friendlyName, attribute, SearchableFieldType.STRING);
-    }
-
-    NamedSearchableField(final String identifier, final String searchableName, final String friendlyName, final boolean attribute, final SearchableFieldType fieldType) {
-        this.identifier = requireNonNull(identifier);
-        this.searchableName = requireNonNull(searchableName);
-        this.friendlyName = requireNonNull(friendlyName);
-        this.attribute = requireNonNull(attribute);
-        this.fieldType = requireNonNull(fieldType);
-    }
-
-    @Override
-    public String getIdentifier() {
-        return identifier;
-    }
-
-    @Override
-    public String getSearchableFieldName() {
-        return searchableName;
-    }
-
-    @Override
-    public String getFriendlyName() {
-        return friendlyName;
-    }
-
-    @Override
-    public boolean isAttribute() {
-        return attribute;
-    }
-
-    @Override
-    public SearchableFieldType getFieldType() {
-        return fieldType;
-    }
-
-    @Override
-    public String toString() {
-        return friendlyName;
-    }
-
-    @Override
-    public int hashCode() {
-        return 298347 + searchableName.hashCode() + (attribute ? 1 : 0);
-    }
-
-    @Override
-    public boolean equals(final Object obj) {
-        if (obj == null) {
-            return false;
-        }
-
-        if (!(obj instanceof SearchableField)) {
-            return false;
-        }
-
-        final SearchableField other = (SearchableField) obj;
-        return (this.searchableName.equals(other.getSearchableFieldName()) && attribute == other.isAttribute());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/SearchableFieldParser.java
----------------------------------------------------------------------
diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/SearchableFieldParser.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/SearchableFieldParser.java
deleted file mode 100644
index 6a934b1..0000000
--- a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/SearchableFieldParser.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.provenance;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.nifi.provenance.search.SearchableField;
-
-public class SearchableFieldParser {
-
-    public static List<SearchableField> extractSearchableFields(final String indexedFieldString, final boolean predefinedField) {
-        final List<SearchableField> searchableFields = new ArrayList<>();
-        if (indexedFieldString != null) {
-            final String[] split = indexedFieldString.split(",");
-            for (String fieldName : split) {
-                fieldName = fieldName.trim();
-                if (fieldName.isEmpty()) {
-                    continue;
-                }
-
-                final SearchableField searchableField;
-                if (predefinedField) {
-                    searchableField = SearchableFields.getSearchableField(fieldName);
-                } else {
-                    searchableField = SearchableFields.newSearchableAttribute(fieldName);
-                }
-
-                if (searchableField == null) {
-                    throw new RuntimeException("Invalid Configuration: Provenance Repository configured to Index field '" + fieldName + "', but this is not a valid field");
-                }
-                searchableFields.add(searchableField);
-            }
-        }
-
-        return searchableFields;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/SearchableFields.java
----------------------------------------------------------------------
diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/SearchableFields.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/SearchableFields.java
deleted file mode 100644
index 97c9880..0000000
--- a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/SearchableFields.java
+++ /dev/null
@@ -1,84 +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.provenance;
-
-import org.apache.nifi.provenance.search.SearchableField;
-import org.apache.nifi.provenance.search.SearchableFieldType;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- *
- */
-public class SearchableFields {
-
-    public static final SearchableField Identifier = new NamedSearchableField("Identifier", "identifier", "Identifier", false);
-    public static final SearchableField EventTime = new NamedSearchableField("EventTime", "time", "Event Time", false, SearchableFieldType.DATE);
-    public static final SearchableField FlowFileUUID = new NamedSearchableField("FlowFileUUID", "uuid", "FlowFile UUID", false);
-    public static final SearchableField Filename = new NamedSearchableField("Filename", "filename", "Filename", false);
-    public static final SearchableField EventType = new NamedSearchableField("EventType", "eventType", "Event Type", false);
-    public static final SearchableField TransitURI = new NamedSearchableField("TransitURI", "transitUri", "Transit URI", false);
-    public static final SearchableField ComponentID = new NamedSearchableField("ProcessorID", "processorId", "Component ID", false);
-    public static final SearchableField AlternateIdentifierURI = new NamedSearchableField("AlternateIdentifierURI", "alternateIdentifierUri", "Alternate Identifier URI", false);
-    public static final SearchableField FileSize = new NamedSearchableField("FileSize", "fileSize", "File Size", false, SearchableFieldType.DATA_SIZE);
-    public static final SearchableField Details = new NamedSearchableField("Details", "details", "Details", false, SearchableFieldType.STRING);
-    public static final SearchableField Relationship = new NamedSearchableField("Relationship", "relationship", "Relationship", false, SearchableFieldType.STRING);
-
-    public static final SearchableField LineageStartDate = new NamedSearchableField("LineageStartDate", "lineageStartDate", "Lineage Start Date", false, SearchableFieldType.DATE);
-    public static final SearchableField LineageIdentifier = new NamedSearchableField("LineageIdentifiers", "lineageIdentifier", "Lineage Identifier", false, SearchableFieldType.STRING);
-
-    public static final SearchableField ContentClaimSection = new NamedSearchableField("ContentClaimSection", "contentClaimSection", "Content Claim Section", false, SearchableFieldType.STRING);
-    public static final SearchableField ContentClaimContainer = new NamedSearchableField("ContentClaimContainer", "contentClaimContainer", "Content Claim Container", false, SearchableFieldType.STRING);
-    public static final SearchableField ContentClaimIdentifier = new NamedSearchableField("ContentClaimIdentifier", "contentClaimIdentifier", "Content Claim Identifier", false, SearchableFieldType.STRING);
-    public static final SearchableField ContentClaimOffset = new NamedSearchableField("ContentClaimOffset", "contentClaimOffset", "Content Claim Offset", false, SearchableFieldType.LONG);
-    public static final SearchableField SourceQueueIdentifier = new NamedSearchableField("SourceQueueIdentifier", "sourceQueueIdentifier", "Source Queue Identifier", false, SearchableFieldType.STRING);
-
-    private static final Map<String, SearchableField> standardFields;
-
-    static {
-        final SearchableField[] searchableFields = new SearchableField[]{
-            EventTime, FlowFileUUID, Filename, EventType, TransitURI,
-            ComponentID, AlternateIdentifierURI, FileSize, Relationship, Details,
-            LineageStartDate, LineageIdentifier, ContentClaimSection, ContentClaimContainer, ContentClaimIdentifier,
-            ContentClaimOffset, SourceQueueIdentifier};
-
-        final Map<String, SearchableField> fields = new HashMap<>();
-        for (final SearchableField field : searchableFields) {
-            fields.put(field.getIdentifier(), field);
-        }
-
-        standardFields = Collections.unmodifiableMap(fields);
-    }
-
-    private SearchableFields() {
-    }
-
-    public static Collection<SearchableField> getStandardFields() {
-        return standardFields.values();
-    }
-
-    public static SearchableField getSearchableField(final String fieldIdentifier) {
-        return standardFields.get(fieldIdentifier);
-    }
-
-    public static SearchableField newSearchableAttribute(final String attributeName) {
-        return new NamedSearchableField(attributeName, attributeName, attributeName, true);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java
----------------------------------------------------------------------
diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java
deleted file mode 100644
index afb56e8..0000000
--- a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java
+++ /dev/null
@@ -1,324 +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.provenance;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-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 java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.nifi.provenance.lineage.ComputeLineageResult;
-import org.apache.nifi.provenance.lineage.EdgeNode;
-import org.apache.nifi.provenance.lineage.EventNode;
-import org.apache.nifi.provenance.lineage.FlowFileNode;
-import org.apache.nifi.provenance.lineage.LineageEdge;
-import org.apache.nifi.provenance.lineage.LineageNode;
-
-/**
- *
- */
-public class StandardLineageResult implements ComputeLineageResult {
-
-    public static final int TTL = (int) TimeUnit.MILLISECONDS.convert(30, TimeUnit.MINUTES);
-    private static final Logger logger = LoggerFactory.getLogger(StandardLineageResult.class);
-
-    private final Collection<String> flowFileUuids;
-    private final Collection<ProvenanceEventRecord> relevantRecords = new ArrayList<>();
-    private final Set<LineageNode> nodes = new HashSet<>();
-    private final Set<LineageEdge> edges = new HashSet<>();
-    private final int numSteps;
-    private final long creationNanos;
-    private long computationNanos;
-
-    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
-    private final Lock readLock = rwLock.readLock();
-    private final Lock writeLock = rwLock.writeLock();
-
-    private Date expirationDate = null;
-    private String error = null;
-    private int numCompletedSteps = 0;
-
-    private volatile boolean canceled = false;
-
-    public StandardLineageResult(final int numSteps, final Collection<String> flowFileUuids) {
-        this.numSteps = numSteps;
-        this.creationNanos = System.nanoTime();
-        this.flowFileUuids = flowFileUuids;
-
-        updateExpiration();
-    }
-
-    @Override
-    public List<LineageNode> getNodes() {
-        readLock.lock();
-        try {
-            return new ArrayList<>(nodes);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public List<LineageEdge> getEdges() {
-        readLock.lock();
-        try {
-            return new ArrayList<>(edges);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    public int getNumberOfEdges() {
-        readLock.lock();
-        try {
-            return edges.size();
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    public int getNumberOfNodes() {
-        readLock.lock();
-        try {
-            return nodes.size();
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    public long getComputationTime(final TimeUnit timeUnit) {
-        readLock.lock();
-        try {
-            return timeUnit.convert(computationNanos, TimeUnit.NANOSECONDS);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public Date getExpiration() {
-        readLock.lock();
-        try {
-            return expirationDate;
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public String getError() {
-        readLock.lock();
-        try {
-            return error;
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public int getPercentComplete() {
-        readLock.lock();
-        try {
-            return (numSteps < 1) ? 100 : (int) (((float) numCompletedSteps / (float) numSteps) * 100.0F);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public boolean isFinished() {
-        readLock.lock();
-        try {
-            return numCompletedSteps >= numSteps || canceled;
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    public void setError(final String error) {
-        writeLock.lock();
-        try {
-            this.error = error;
-            numCompletedSteps++;
-
-            updateExpiration();
-
-            if (numCompletedSteps >= numSteps) {
-                computationNanos = System.nanoTime() - creationNanos;
-            }
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    public void update(final Collection<ProvenanceEventRecord> records) {
-        writeLock.lock();
-        try {
-            relevantRecords.addAll(records);
-
-            numCompletedSteps++;
-            updateExpiration();
-
-            if (numCompletedSteps >= numSteps && error == null) {
-                computeLineage();
-                computationNanos = System.nanoTime() - creationNanos;
-            }
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * Computes the lineage from the relevant Provenance Event Records. This
-     * method must be called with the write lock held and is only going to be
-     * useful after all of the records have been successfully obtained
-     */
-    private void computeLineage() {
-        final long startNanos = System.nanoTime();
-
-        nodes.clear();
-        edges.clear();
-
-        Map<String, LineageNode> lastEventMap = new HashMap<>();    // maps FlowFile UUID to last event for that FlowFile
-        final List<ProvenanceEventRecord> sortedRecords = new ArrayList<>(relevantRecords);
-        Collections.sort(sortedRecords, new Comparator<ProvenanceEventRecord>() {
-            @Override
-            public int compare(final ProvenanceEventRecord o1, final ProvenanceEventRecord o2) {
-                // Sort on Event Time, then Event ID.
-                final int eventTimeComparison = Long.compare(o1.getEventTime(), o2.getEventTime());
-                if (eventTimeComparison == 0) {
-                    return Long.compare(o1.getEventId(), o2.getEventId());
-                } else {
-                    return eventTimeComparison;
-                }
-            }
-        });
-
-        // convert the StandardProvenanceRecord objects into Lineage nodes (FlowFileNode, EventNodes).
-        for (final ProvenanceEventRecord record : sortedRecords) {
-            final LineageNode lineageNode = new EventNode(record);
-            final boolean added = nodes.add(lineageNode);
-            if (!added) {
-                logger.debug("Did not add {} because it already exists in the 'nodes' set", lineageNode);
-            }
-
-            // Create an edge that connects this node to the previous node for the same FlowFile UUID.
-            final LineageNode lastNode = lastEventMap.get(record.getFlowFileUuid());
-            if (lastNode != null) {
-                // We calculate the Edge UUID based on whether or not this event is a SPAWN.
-                // If this event is a SPAWN, then we want to use the previous node's UUID because a
-                // SPAWN Event's UUID is not necessarily what we want, since a SPAWN Event's UUID pertains to
-                // only one of (potentially) many UUIDs associated with the event. Otherwise, we know that
-                // the UUID of this record is appropriate, so we just use it.
-                final String edgeUuid;
-
-                switch (record.getEventType()) {
-                    case JOIN:
-                    case CLONE:
-                    case REPLAY:
-                        edgeUuid = lastNode.getFlowFileUuid();
-                        break;
-                    default:
-                        edgeUuid = record.getFlowFileUuid();
-                        break;
-                }
-
-                edges.add(new EdgeNode(edgeUuid, lastNode, lineageNode));
-            }
-
-            lastEventMap.put(record.getFlowFileUuid(), lineageNode);
-
-            switch (record.getEventType()) {
-                case FORK:
-                case JOIN:
-                case REPLAY:
-                case CLONE: {
-                    // For events that create FlowFile nodes, we need to create the FlowFile Nodes and associated Edges, as appropriate
-                    for (final String childUuid : record.getChildUuids()) {
-                        if (flowFileUuids.contains(childUuid)) {
-                            final FlowFileNode childNode = new FlowFileNode(childUuid, record.getEventTime());
-                            final boolean isNewFlowFile = nodes.add(childNode);
-                            if (!isNewFlowFile) {
-                                final String msg = "Unable to generate Lineage Graph because multiple events were registered claiming to have generated the same FlowFile (UUID = " + childNode.getFlowFileUuid() + ")";
-                                logger.error(msg);
-                                setError(msg);
-                                return;
-                            }
-
-                            edges.add(new EdgeNode(childNode.getFlowFileUuid(), lineageNode, childNode));
-                            lastEventMap.put(childUuid, childNode);
-                        }
-                    }
-                    for (final String parentUuid : record.getParentUuids()) {
-                        LineageNode lastNodeForParent = lastEventMap.get(parentUuid);
-                        if (lastNodeForParent != null && !lastNodeForParent.equals(lineageNode)) {
-                            edges.add(new EdgeNode(parentUuid, lastNodeForParent, lineageNode));
-                        }
-
-                        lastEventMap.put(parentUuid, lineageNode);
-                    }
-                }
-                break;
-                case RECEIVE:
-                case CREATE: {
-                        // for a receive event, we want to create a FlowFile Node that represents the FlowFile received
-                    // and create an edge from the Receive Event to the FlowFile Node
-                    final LineageNode flowFileNode = new FlowFileNode(record.getFlowFileUuid(), record.getEventTime());
-                    final boolean isNewFlowFile = nodes.add(flowFileNode);
-                    if (!isNewFlowFile) {
-                        final String msg = "Found cycle in graph. This indicates that multiple events were registered claiming to have generated the same FlowFile (UUID = " + flowFileNode.getFlowFileUuid() + ")";
-                        setError(msg);
-                        logger.error(msg);
-                        return;
-                    }
-                    edges.add(new EdgeNode(record.getFlowFileUuid(), lineageNode, flowFileNode));
-                    lastEventMap.put(record.getFlowFileUuid(), flowFileNode);
-                }
-                break;
-                default:
-                    break;
-            }
-        }
-
-        final long nanos = System.nanoTime() - startNanos;
-        logger.debug("Finished building lineage with {} nodes and {} edges in {} millis", nodes.size(), edges.size(), TimeUnit.NANOSECONDS.toMillis(nanos));
-    }
-
-    void cancel() {
-        this.canceled = true;
-    }
-
-    /**
-     * Must be called with write lock!
-     */
-    private void updateExpiration() {
-        expirationDate = new Date(System.currentTimeMillis() + TTL);
-    }
-}


[16/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/ReflexiveProtocolHandler.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/ReflexiveProtocolHandler.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/ReflexiveProtocolHandler.java
deleted file mode 100644
index 4e3b932..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/ReflexiveProtocolHandler.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.cluster.protocol.impl.testutils;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-
-/**
- * @author unattributed
- */
-public class ReflexiveProtocolHandler implements ProtocolHandler {
-    
-    private List<ProtocolMessage> messages = new ArrayList<>();
-        
-    @Override
-    public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
-        messages.add(msg);
-        return msg;
-    }
-
-    @Override
-    public boolean canHandle(ProtocolMessage msg) {
-        return true;
-    }
-
-    public List<ProtocolMessage> getMessages() {
-        return messages;
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-web/.gitignore
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-web/.gitignore b/nar-bundles/framework-bundle/framework/cluster-web/.gitignore
deleted file mode 100755
index ea8c4bf..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-web/.gitignore
+++ /dev/null
@@ -1 +0,0 @@
-/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-web/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-web/pom.xml b/nar-bundles/framework-bundle/framework/cluster-web/pom.xml
deleted file mode 100644
index a7c39c6..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-web/pom.xml
+++ /dev/null
@@ -1,50 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-framework-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-    <artifactId>framework-cluster-web</artifactId>
-    <packaging>jar</packaging>
-    <name>NiFi Framework Cluster Web</name>
-    <description>The clustering software for communicating with the NiFi web api.</description>
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-properties</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>web-optimistic-locking</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-administration</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-user-actions</artifactId>
-        </dependency>
-        
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContext.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContext.java b/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContext.java
deleted file mode 100644
index 44fb25a..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContext.java
+++ /dev/null
@@ -1,59 +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.cluster.context;
-
-import java.io.Serializable;
-import java.util.List;
-import org.apache.nifi.action.Action;
-import org.apache.nifi.web.Revision;
-
-/**
- * Contains contextual information about clustering that may be serialized 
- * between manager and node when communicating over HTTP.
- */
-public interface ClusterContext extends Serializable {
-    
-    /**
-     * Returns a list of auditable actions.  The list is modifiable
-     * and will never be null.
-     * @return a collection of actions
-     */
-    List<Action> getActions();
-    
-    Revision getRevision();
-    
-    void setRevision(Revision revision);
-    
-    /**
-     * @return true if the request was sent by the cluster manager; false otherwise
-     */
-    boolean isRequestSentByClusterManager();
-    
-    /**
-     * Sets the flag to indicate if a request was sent by the cluster manager.
-     * @param flag true if the request was sent by the cluster manager; false otherwise
-     */
-    void setRequestSentByClusterManager(boolean flag);
-    
-    /**
-     * Gets an id generation seed. This is used to ensure that nodes are able to generate the
-     * same id across the cluster. This is usually handled by the cluster manager creating the
-     * id, however for some actions (snippets, templates, etc) this is not possible.
-     * @return 
-     */
-    String getIdGenerationSeed();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextImpl.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextImpl.java b/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextImpl.java
deleted file mode 100644
index 06907d2..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextImpl.java
+++ /dev/null
@@ -1,69 +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.cluster.context;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.UUID;
-import org.apache.nifi.action.Action;
-import org.apache.nifi.web.Revision;
-
-/**
- * A basic implementation of the context.
- */
-public class ClusterContextImpl implements ClusterContext, Serializable {
-
-    private final List<Action> actions = new ArrayList<>();
-    
-    private Revision revision;
-    
-    private boolean requestSentByClusterManager;
-    
-    private final String idGenerationSeed = UUID.randomUUID().toString();
-    
-    @Override
-    public List<Action> getActions() {
-        return actions;
-    }
-
-    @Override
-    public Revision getRevision() {
-        return revision;
-    }
-
-    @Override
-    public void setRevision(Revision revision) {
-        this.revision = revision;
-    }
-
-    @Override
-    public boolean isRequestSentByClusterManager() {
-        return requestSentByClusterManager;
-    }
-    
-    @Override
-    public void setRequestSentByClusterManager(boolean requestSentByClusterManager) {
-        this.requestSentByClusterManager = requestSentByClusterManager;
-    }
-
-    @Override
-    public String getIdGenerationSeed() {
-        return this.idGenerationSeed;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextThreadLocal.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextThreadLocal.java b/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextThreadLocal.java
deleted file mode 100644
index 012e7c7..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextThreadLocal.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.cluster.context;
-
-/**
- * Manages a cluster context on a threadlocal.
- */
-public class ClusterContextThreadLocal {
-    
-    private static final ThreadLocal<ClusterContext> contextHolder = new ThreadLocal<>();
-    
-    public static void removeContext() {
-        contextHolder.remove();
-    }
-    
-    public static ClusterContext createEmptyContext() {
-        return new ClusterContextImpl();
-    }
-    
-    public static ClusterContext getContext() {
-        ClusterContext ctx = contextHolder.get();
-        if(ctx == null) {
-            ctx = createEmptyContext();
-            contextHolder.set(ctx);
-        }
-        return ctx;
-    }
-    
-    public static void setContext(final ClusterContext context) {
-        contextHolder.set(context);
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java b/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java
deleted file mode 100644
index 90b8a37..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java
+++ /dev/null
@@ -1,96 +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.cluster.context.ClusterContext;
-import org.apache.nifi.cluster.context.ClusterContextThreadLocal;
-
-/**
- * An optimistic locking manager that provides for optimistic locking in a clustered
- * environment.
- * 
- * @author unattributed
- */
-public class ClusterAwareOptimisticLockingManager implements OptimisticLockingManager {
-
-    private final OptimisticLockingManager optimisticLockingManager;
-    
-    public ClusterAwareOptimisticLockingManager(final OptimisticLockingManager optimisticLockingManager) {
-        this.optimisticLockingManager = optimisticLockingManager;
-    }
-    
-    @Override
-    public Revision checkRevision(Revision revision) throws InvalidRevisionException {
-        final Revision currentRevision = getRevision();
-        if(currentRevision.equals(revision) == false) {
-            throw new InvalidRevisionException(String.format("Given revision %s does not match current revision %s.", revision, currentRevision));
-        } else {
-            return revision.increment(revision.getClientId());
-        }
-    }
-
-    @Override
-    public boolean isCurrent(Revision revision) {
-        return getRevision().equals(revision);
-    }
-
-    @Override
-    public Revision getRevision() {
-        final ClusterContext ctx = ClusterContextThreadLocal.getContext();
-        if(ctx == null || ctx.getRevision() == null) {
-            return optimisticLockingManager.getRevision();
-        } else {
-            return ctx.getRevision();
-        }
-    }
-
-    @Override
-    public void setRevision(final Revision revision) {
-        final ClusterContext ctx = ClusterContextThreadLocal.getContext();
-        if(ctx != null) {
-            ctx.setRevision(revision);
-        }
-        optimisticLockingManager.setRevision(revision);
-    }
-
-    @Override
-    public Revision incrementRevision() {
-        final Revision currentRevision = getRevision();
-        final Revision incRevision = currentRevision.increment();
-        setRevision(incRevision);
-        return incRevision;
-    }
-
-    @Override
-    public Revision incrementRevision(final String clientId) {
-        final Revision currentRevision = getRevision();
-        final Revision incRevision = currentRevision.increment(clientId);
-        setRevision(incRevision);
-        return incRevision;
-    }
-
-    @Override
-    public String getLastModifier() {
-        return optimisticLockingManager.getLastModifier();
-    }
-
-    @Override
-    public void setLastModifier(final String lastModifier) {
-        optimisticLockingManager.setLastModifier(lastModifier);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/.gitignore
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/.gitignore b/nar-bundles/framework-bundle/framework/cluster/.gitignore
deleted file mode 100755
index ea8c4bf..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/.gitignore
+++ /dev/null
@@ -1 +0,0 @@
-/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/pom.xml b/nar-bundles/framework-bundle/framework/cluster/pom.xml
deleted file mode 100644
index 6712802..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/pom.xml
+++ /dev/null
@@ -1,132 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-framework-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-    <artifactId>framework-cluster</artifactId>
-    <packaging>jar</packaging>
-    <name>NiFi Framework Cluster</name>
-    <description>The clustering software for NiFi.</description>
-    <dependencies>
-        
-        <!-- application core dependencies -->
-        
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-properties</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-logging-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>client-dto</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>framework-core</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>core-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>framework-cluster-protocol</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>framework-cluster-web</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-web-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-administration</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>site-to-site</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-compress</artifactId>
-        </dependency>
-        
-        <!-- third party dependencies -->
-        
-        <!-- sun dependencies -->
-        <dependency>
-            <groupId>javax.servlet</groupId>
-            <artifactId>javax.servlet-api</artifactId>
-        </dependency>
-        
-        <!-- commons dependencies -->
-        <dependency>
-            <groupId>commons-io</groupId>
-            <artifactId>commons-io</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-net</groupId>
-            <artifactId>commons-net</artifactId>
-        </dependency>
-        
-        <!-- jersey dependencies -->
-        <dependency>
-            <groupId>com.sun.jersey</groupId>
-            <artifactId>jersey-client</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.sun.jersey</groupId>
-            <artifactId>jersey-server</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.sun.jersey</groupId>
-            <artifactId>jersey-json</artifactId>
-        </dependency>
-        
-        <!-- spring dependencies -->
-        <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-core</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-beans</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-context</artifactId>
-        </dependency>
-        
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java
deleted file mode 100644
index 0b70c61..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java
+++ /dev/null
@@ -1,151 +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.cluster.client;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.net.DatagramPacket;
-import java.net.InetSocketAddress;
-import java.net.MulticastSocket;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Locale;
-import java.util.Timer;
-import java.util.TimerTask;
-
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.impl.MulticastProtocolListener;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.PingMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.apache.nifi.io.socket.multicast.MulticastUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Simple multicast test client that sends ping messages to a group address.
- */
-public class MulticastTestClient {
-
-    private static final Logger logger = LoggerFactory.getLogger(MulticastTestClient.class);
-
-    private static final int PING_DELAY_SECONDS = 3;
-
-    public static void main(final String... args) throws IOException {
-
-        String group = System.getProperty("group", "225.0.0.0");
-        if (group == null) {
-            System.out.println("Host system property 'group' was not given.");
-            return;
-        }
-        group = group.trim();
-        if (group.length() == 0) {
-            System.out.println("Host system property 'group' must be non-empty.");
-            return;
-        }
-
-        final String portStr = System.getProperty("port", "2222");
-        final int port;
-        try {
-            port = Integer.parseInt(portStr);
-        } catch (final NumberFormatException nfe) {
-            System.out.println("Port system property 'port' was not a valid port.");
-            return;
-        }
-
-        logger.info(String.format("Pinging every %s seconds using multicast address: %s:%s.", PING_DELAY_SECONDS, group, port));
-        logger.info("Override defaults by using system properties '-Dgroup=<Class D IP>' and '-Dport=<unused port>'.");
-        logger.info("The test client may be stopped by entering a newline at the command line.");
-
-        final InetSocketAddress addr = new InetSocketAddress(group, port);
-        final ProtocolContext<ProtocolMessage> protocolContext = new JaxbProtocolContext<ProtocolMessage>(JaxbProtocolUtils.JAXB_CONTEXT);
-        final MulticastConfiguration multicastConfig = new MulticastConfiguration();
-        multicastConfig.setReuseAddress(true);
-
-        // setup listener
-        final MulticastProtocolListener listener = new MulticastProtocolListener(1, addr, multicastConfig, protocolContext);
-        listener.addHandler(new ProtocolHandler() {
-            @Override
-            public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
-                final PingMessage pingMsg = (PingMessage) msg;
-                final SimpleDateFormat sdf = new SimpleDateFormat("HH:mm:ss", Locale.US);
-                logger.info("Pinged at: " + sdf.format(pingMsg.getDate()));
-                return null;
-            }
-
-            @Override
-            public boolean canHandle(ProtocolMessage msg) {
-                return true;
-            }
-        });
-
-        // setup socket
-        final MulticastSocket multicastSocket = MulticastUtils.createMulticastSocket(multicastConfig);
-
-        // setup broadcaster
-        final Timer broadcaster = new Timer("Multicast Test Client", /**
-                 * is daemon *
-                 */
-                true);
-
-        try {
-
-            // start listening
-            listener.start();
-
-            // start broadcasting
-            broadcaster.schedule(new TimerTask() {
-
-                @Override
-                public void run() {
-                    try {
-
-                        final PingMessage msg = new PingMessage();
-                        msg.setDate(new Date());
-
-                        // marshal message to output stream
-                        final ProtocolMessageMarshaller<ProtocolMessage> marshaller = protocolContext.createMarshaller();
-                        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-                        marshaller.marshal(msg, baos);
-                        final byte[] packetBytes = baos.toByteArray();
-
-                        // send message
-                        final DatagramPacket packet = new DatagramPacket(packetBytes, packetBytes.length, addr);
-                        multicastSocket.send(packet);
-
-                    } catch (final Exception ex) {
-                        logger.warn("Failed to send message due to: " + ex, ex);
-                    }
-                }
-            }, 0, PING_DELAY_SECONDS * 1000);
-
-            // block until any input is received
-            System.in.read();
-
-        } finally {
-            broadcaster.cancel();
-            if (listener.isRunning()) {
-                listener.stop();
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/Event.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/Event.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/Event.java
deleted file mode 100644
index 6bc5d6c..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/Event.java
+++ /dev/null
@@ -1,122 +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.cluster.event;
-
-import java.util.Date;
-import org.apache.commons.lang3.StringUtils;
-
-/**
- * Events describe the occurrence of something noteworthy. They record the
- * event's source, a timestamp, a description, and a category.
- *
- * @author unattributed
- *
- * @Immutable
- */
-public class Event {
-
-    public static enum Category {
-
-        DEBUG,
-        INFO,
-        WARN
-    }
-
-    private final String source;
-
-    private final long timestamp;
-
-    private final Category category;
-
-    private final String message;
-
-    /**
-     * Creates an event with the current time as the timestamp and a category of
-     * "INFO".
-     *
-     * @param source the source
-     * @param message the description
-     */
-    public Event(final String source, final String message) {
-        this(source, message, Category.INFO);
-    }
-
-    /**
-     * Creates an event with the current time as the timestamp.
-     *
-     * @param source the source
-     * @param message the description
-     * @param category the event category
-     */
-    public Event(final String source, final String message, final Category category) {
-        this(source, message, category, new Date().getTime());
-    }
-
-    /**
-     * Creates an event with the a category of "INFO".
-     *
-     * @param source the source
-     * @param message the description
-     * @param timestamp the time of occurrence
-     */
-    public Event(final String source, final String message, final long timestamp) {
-        this(source, message, Category.INFO, timestamp);
-    }
-
-    /**
-     * Creates an event.
-     *
-     * @param source the source
-     * @param message the description
-     * @param category the event category
-     * @param timestamp the time of occurrence
-     */
-    public Event(final String source, final String message, final Category category, final long timestamp) {
-
-        if (StringUtils.isBlank(source)) {
-            throw new IllegalArgumentException("Source may not be empty or null.");
-        } else if (StringUtils.isBlank(message)) {
-            throw new IllegalArgumentException("Event message may not be empty or null.");
-        } else if (category == null) {
-            throw new IllegalArgumentException("Event category may not be null.");
-        } else if (timestamp < 0) {
-            throw new IllegalArgumentException("Timestamp may not be negative: " + timestamp);
-        }
-
-        this.source = source;
-        this.message = message;
-        this.category = category;
-        this.timestamp = timestamp;
-    }
-
-    public Category getCategory() {
-        return category;
-    }
-
-    public String getMessage() {
-        return message;
-    }
-
-    public String getSource() {
-        return source;
-    }
-
-    public long getTimestamp() {
-        return timestamp;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/EventManager.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/EventManager.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/EventManager.java
deleted file mode 100644
index f9dfb00..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/EventManager.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.cluster.event;
-
-import java.util.List;
-
-/**
- * Manages an ordered list of events. The event history size dictates the total
- * number of events to manage for a given source at a given time. When the size
- * is exceeded, the oldest event for that source is evicted.
- *
- * @author unattributed
- */
-public interface EventManager {
-
-    /**
-     * Adds an event to the manager.
-     *
-     * @param event an Event
-     */
-    void addEvent(Event event);
-
-    /**
-     * Returns a list of events for a given source sorted by the event's
-     * timestamp where the most recent event is first in the list.
-     *
-     * @param eventSource the source
-     *
-     * @return the list of events
-     */
-    List<Event> getEvents(String eventSource);
-
-    /*
-     * Returns the most recent event for the source.  If no events exist, then
-     * null is returned.
-     */
-    Event getMostRecentEvent(String eventSource);
-
-    /*
-     * Clears all events for the given source.
-     */
-    void clearEventHistory(String eventSource);
-
-    /**
-     * Returns the history size.
-     *
-     * @return the history size
-     */
-    int getEventHistorySize();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/impl/EventManagerImpl.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/impl/EventManagerImpl.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/impl/EventManagerImpl.java
deleted file mode 100644
index 7fadc78..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/impl/EventManagerImpl.java
+++ /dev/null
@@ -1,143 +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.cluster.event.impl;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.PriorityQueue;
-import java.util.Queue;
-import org.apache.nifi.cluster.event.Event;
-import org.apache.nifi.cluster.event.EventManager;
-
-/**
- * Implements the EventManager.
- *
- * @author unattributed
- */
-public class EventManagerImpl implements EventManager {
-
-    /**
-     * associates the source ID with an ordered queue of events, ordered by most
-     * recent event
-     */
-    private final Map<String, Queue<Event>> eventsMap = new HashMap<>();
-
-    /**
-     * the number of events to maintain for a given source
-     */
-    private final int eventHistorySize;
-
-    /**
-     * Creates an instance.
-     *
-     * @param eventHistorySize the number of events to manage for a given
-     * source. Value must be positive.
-     */
-    public EventManagerImpl(final int eventHistorySize) {
-        if (eventHistorySize <= 0) {
-            throw new IllegalArgumentException("Event history size must be positive: " + eventHistorySize);
-        }
-        this.eventHistorySize = eventHistorySize;
-    }
-
-    @Override
-    public void addEvent(final Event event) {
-
-        if (event == null) {
-            throw new IllegalArgumentException("Event may not be null.");
-        }
-
-        Queue<Event> events = eventsMap.get(event.getSource());
-        if (events == null) {
-            // no events from this source, so add a new queue to the map
-            events = new PriorityQueue<>(eventHistorySize, createEventComparator());
-            eventsMap.put(event.getSource(), events);
-        }
-
-        // add event
-        events.add(event);
-
-        // if we exceeded the history size, then evict the oldest event
-        if (events.size() > eventHistorySize) {
-            removeOldestEvent(events);
-        }
-
-    }
-
-    @Override
-    public List<Event> getEvents(final String eventSource) {
-        final Queue<Event> events = eventsMap.get(eventSource);
-        if (events == null) {
-            return Collections.EMPTY_LIST;
-        } else {
-            return Collections.unmodifiableList(new ArrayList<>(events));
-        }
-    }
-
-    @Override
-    public int getEventHistorySize() {
-        return eventHistorySize;
-    }
-
-    @Override
-    public Event getMostRecentEvent(final String eventSource) {
-        final Queue<Event> events = eventsMap.get(eventSource);
-        if (events == null) {
-            return null;
-        } else {
-            return events.peek();
-        }
-    }
-
-    @Override
-    public void clearEventHistory(final String eventSource) {
-        eventsMap.remove(eventSource);
-    }
-
-    private Comparator createEventComparator() {
-        return new Comparator<Event>() {
-            @Override
-            public int compare(final Event o1, final Event o2) {
-                // orders events by most recent first
-                return (int) (o2.getTimestamp() - o1.getTimestamp());
-            }
-        };
-    }
-
-    private void removeOldestEvent(final Collection<Event> events) {
-
-        if (events.isEmpty()) {
-            return;
-        }
-
-        Event oldestEvent = null;
-        for (final Event event : events) {
-            if (oldestEvent == null || oldestEvent.getTimestamp() > event.getTimestamp()) {
-                oldestEvent = event;
-            }
-        }
-
-        events.remove(oldestEvent);
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/firewall/ClusterNodeFirewall.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/firewall/ClusterNodeFirewall.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/firewall/ClusterNodeFirewall.java
deleted file mode 100644
index 2e3d278..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/firewall/ClusterNodeFirewall.java
+++ /dev/null
@@ -1,35 +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.cluster.firewall;
-
-/**
- * Defines the interface for restricting external client connections to a set of
- * hosts or IPs.
- */
-public interface ClusterNodeFirewall {
-
-    /**
-     * Returns true if the given host or IP is permissible through the firewall;
-     * false otherwise.
-     *
-     * If an IP is given, then it must be formatted in dotted decimal notation.
-     * @param hostOrIp
-     * @return 
-     */
-    boolean isPermissible(String hostOrIp);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewall.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewall.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewall.java
deleted file mode 100644
index 916ec14..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewall.java
+++ /dev/null
@@ -1,207 +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.cluster.firewall.impl;
-
-import java.io.*;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.*;
-import org.apache.commons.net.util.SubnetUtils;
-import org.apache.nifi.cluster.firewall.ClusterNodeFirewall;
-import org.apache.nifi.util.file.FileUtils;
-import org.apache.nifi.logging.NiFiLog;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A file-based implementation of the ClusterFirewall interface. The class is
- * configured with a file. If the file is empty, then everything is permissible.
- * Otherwise, the file should contain hostnames or IPs formatted as dotted
- * decimals with an optional CIDR suffix. Each entry must be separated by a
- * newline. An example configuration is given below:
- *
- * <code>
- * # hash character is a comment delimiter
- * 1.2.3.4         # exact IP
- * some.host.name  # a host name
- * 4.5.6.7/8       # range of CIDR IPs
- * 9.10.11.12/13   # a smaller range of CIDR IPs
- * </code>
- *
- * This class allows for synchronization with an optionally configured restore
- * directory. If configured, then at startup, if the either the config file or
- * the restore directory's copy is missing, then the configuration file will be
- * copied to the appropriate location. If both restore directory contains a copy
- * that is different in content to configuration file, then an exception is
- * thrown at construction time.
- */
-public class FileBasedClusterNodeFirewall implements ClusterNodeFirewall {
-
-    private final File config;
-
-    private final File restoreDirectory;
-
-    private final Collection<SubnetUtils.SubnetInfo> subnetInfos = new ArrayList<>();
-
-    private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(FileBasedClusterNodeFirewall.class));
-
-    public FileBasedClusterNodeFirewall(final File config) throws IOException {
-        this(config, null);
-    }
-
-    public FileBasedClusterNodeFirewall(final File config, final File restoreDirectory) throws IOException {
-
-        if (config == null) {
-            throw new IllegalArgumentException("Firewall configuration file may not be null.");
-        }
-
-        this.config = config;
-        this.restoreDirectory = restoreDirectory;
-
-        if (restoreDirectory != null) {
-            // synchronize with restore directory
-            try {
-                syncWithRestoreDirectory();
-            } catch (final IOException ioe) {
-                throw new RuntimeException(ioe);
-            }
-        }
-
-        if (!config.exists() && !config.createNewFile()) {
-            throw new IOException("Firewall configuration file did not exist and could not be created: " + config.getAbsolutePath());
-        }
-
-        logger.info("Loading cluster firewall configuration.");
-        parseConfig(config);
-        logger.info("Cluster firewall configuration loaded.");
-    }
-
-    @Override
-    public boolean isPermissible(final String hostOrIp) {
-        try {
-
-            // if no rules, then permit everything
-            if (subnetInfos.isEmpty()) {
-                return true;
-            }
-
-            final String ip;
-            try {
-                ip = InetAddress.getByName(hostOrIp).getHostAddress();
-            } catch (final UnknownHostException uhe) {
-                logger.warn("Blocking unknown host: " + hostOrIp, uhe);
-                return false;
-            }
-
-            // check each subnet to see if IP is in range
-            for (final SubnetUtils.SubnetInfo subnetInfo : subnetInfos) {
-                if (subnetInfo.isInRange(ip)) {
-                    return true;
-                }
-            }
-
-            // no match
-            return false;
-
-        } catch (final IllegalArgumentException iae) {
-            return false;
-        }
-    }
-
-    private void syncWithRestoreDirectory() throws IOException {
-
-        // sanity check that restore directory is a directory, creating it if necessary
-        FileUtils.ensureDirectoryExistAndCanAccess(restoreDirectory);
-
-        // check that restore directory is not the same as the primary directory
-        if (config.getParentFile().getAbsolutePath().equals(restoreDirectory.getAbsolutePath())) {
-            throw new IllegalStateException(
-                    String.format("Cluster firewall configuration file '%s' cannot be in the restore directory '%s' ",
-                            config.getAbsolutePath(), restoreDirectory.getAbsolutePath()));
-        }
-
-        // the restore copy will have same file name, but reside in a different directory
-        final File restoreFile = new File(restoreDirectory, config.getName());
-
-        // sync the primary copy with the restore copy
-        FileUtils.syncWithRestore(config, restoreFile, logger);
-
-    }
-
-    private void parseConfig(final File config) throws IOException {
-
-        // clear old information
-        subnetInfos.clear();
-        try (BufferedReader br = new BufferedReader(new FileReader(config))) {
-
-            String ipOrHostLine;
-            String ipCidr;
-            int totalIpsAdded = 0;
-            while ((ipOrHostLine = br.readLine()) != null) {
-
-                // cleanup whitespace
-                ipOrHostLine = ipOrHostLine.trim();
-
-                if (ipOrHostLine.isEmpty() || ipOrHostLine.startsWith("#")) {
-                    // skip empty lines or comments
-                    continue;
-                } else if (ipOrHostLine.contains("#")) {
-                    // parse out comments in IP containing lines
-                    ipOrHostLine = ipOrHostLine.substring(0, ipOrHostLine.indexOf("#")).trim();
-                }
-
-                // if given a complete IP, then covert to CIDR
-                if (ipOrHostLine.contains("/")) {
-                    ipCidr = ipOrHostLine;
-                } else if (ipOrHostLine.contains("\\")) {
-                    logger.warn("CIDR IP notation uses forward slashes '/'.  Replacing backslash '\\' with forward slash'/' for '" + ipOrHostLine + "'");
-                    ipCidr = ipOrHostLine.replace("\\", "/");
-                } else {
-                    try {
-                        ipCidr = InetAddress.getByName(ipOrHostLine).getHostAddress();
-                        if (!ipOrHostLine.equals(ipCidr)) {
-                            logger.debug(String.format("Resolved host '%s' to ip '%s'", ipOrHostLine, ipCidr));
-                        }
-                        ipCidr += "/32";
-                        logger.debug("Adding CIDR to exact IP: " + ipCidr);
-                    } catch (final UnknownHostException uhe) {
-                        logger.warn("Firewall is skipping unknown host address: " + ipOrHostLine);
-                        continue;
-                    }
-                }
-
-                try {
-                    logger.debug("Adding CIDR IP to firewall: " + ipCidr);
-                    final SubnetUtils subnetUtils = new SubnetUtils(ipCidr);
-                    subnetUtils.setInclusiveHostCount(true);
-                    subnetInfos.add(subnetUtils.getInfo());
-                    totalIpsAdded++;
-                } catch (final IllegalArgumentException iae) {
-                    logger.warn("Firewall is skipping invalid CIDR address: " + ipOrHostLine);
-                }
-
-            }
-
-            if (totalIpsAdded == 0) {
-                logger.info("No IPs added to firewall.  Firewall will accept all requests.");
-            } else {
-                logger.info(String.format("Added %d IP(s) to firewall.  Only requests originating from the configured IPs will be accepted.", totalIpsAdded));
-            }
-
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java
deleted file mode 100644
index eedb88f..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java
+++ /dev/null
@@ -1,45 +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.cluster.flow;
-
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.StandardDataFlow;
-
-/**
- * A dataflow with additional information about the cluster.
- *
- * @author unattributed
- */
-public class ClusterDataFlow {
-
-    private final StandardDataFlow dataFlow;
-
-    private final NodeIdentifier primaryNodeId;
-
-    public ClusterDataFlow(final StandardDataFlow dataFlow, final NodeIdentifier primaryNodeId) {
-        this.dataFlow = dataFlow;
-        this.primaryNodeId = primaryNodeId;
-    }
-
-    public NodeIdentifier getPrimaryNodeId() {
-        return primaryNodeId;
-    }
-
-    public StandardDataFlow getDataFlow() {
-        return dataFlow;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.java
deleted file mode 100644
index 6ff15a7..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.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.cluster.flow;
-
-/**
- * A base exception for data access exceptions.
- *
- * @author unattributed
- */
-public class DaoException extends RuntimeException {
-
-    public DaoException() {
-    }
-
-    public DaoException(String msg) {
-        super(msg);
-    }
-
-    public DaoException(Throwable cause) {
-        super(cause);
-    }
-
-    public DaoException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowDao.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowDao.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowDao.java
deleted file mode 100644
index a273704..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowDao.java
+++ /dev/null
@@ -1,62 +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.cluster.flow;
-
-/**
- * A data access object for loading and saving the flow managed by the cluster.
- *
- * @author unattributed
- */
-public interface DataFlowDao {
-
-    /**
-     * Loads the cluster's dataflow.
-     *
-     * @return the dataflow or null if no dataflow exists
-     *
-     * @throws DaoException if the dataflow was unable to be loaded
-     */
-    ClusterDataFlow loadDataFlow() throws DaoException;
-
-    /**
-     * Saves the cluster's dataflow.
-     *
-     *
-     * @param dataFlow
-     * @throws DaoException if the dataflow was unable to be saved
-     */
-    void saveDataFlow(ClusterDataFlow dataFlow) throws DaoException;
-
-    /**
-     * Sets the state of the dataflow. If the dataflow does not exist, then an
-     * exception is thrown.
-     *
-     * @param flowState the state of the dataflow
-     *
-     * @throws DaoException if the state was unable to be updated
-     */
-    void setPersistedFlowState(PersistedFlowState flowState) throws DaoException;
-
-    /**
-     * Gets the state of the dataflow.
-     *
-     * @return the state of the dataflow
-     *
-     * @throws DaoException if the state was unable to be retrieved
-     */
-    PersistedFlowState getPersistedFlowState() throws DaoException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java
deleted file mode 100644
index 339d904..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java
+++ /dev/null
@@ -1,115 +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.cluster.flow;
-
-import java.util.Set;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-
-/**
- * A service for managing the cluster's flow. The service will attempt to keep
- * the cluster's dataflow current while respecting the value of the configured
- * retrieval delay.
- *
- * The eligible retrieval time is reset with the configured delay every time the
- * flow state is set to STALE. If the state is set to UNKNOWN or CURRENT, then
- * the flow will not be retrieved.
- *
- * Clients must call start() and stop() to initialize and stop the instance.
- *
- * @author unattributed
- */
-public interface DataFlowManagementService {
-
-    /**
-     * Starts the instance. Start may only be called if the instance is not
-     * running.
-     */
-    void start();
-
-    /**
-     * Stops the instance. Stop may only be called if the instance is running.
-     */
-    void stop();
-
-    /**
-     * @return true if the instance is started; false otherwise.
-     */
-    boolean isRunning();
-
-    /**
-     * Loads the dataflow.
-     *
-     * @return the dataflow or null if no dataflow exists
-     */
-    ClusterDataFlow loadDataFlow();
-
-    /**
-     * Updates the dataflow with the given primary node identifier.
-     *
-     * @param nodeId the node identifier
-     *
-     * @throws DaoException if the update failed
-     */
-    void updatePrimaryNode(NodeIdentifier nodeId) throws DaoException;
-
-    /**
-     * Sets the state of the flow.
-     *
-     * @param flowState the state
-     *
-     * @see PersistedFlowState
-     */
-    void setPersistedFlowState(PersistedFlowState flowState);
-
-    /**
-     * @return the state of the flow
-     */
-    PersistedFlowState getPersistedFlowState();
-
-    /**
-     * @return true if the flow is current; false otherwise.
-     */
-    boolean isFlowCurrent();
-
-    /**
-     * Sets the node identifiers to use when attempting to retrieve the flow.
-     *
-     * @param nodeIds the node identifiers
-     */
-    void setNodeIds(Set<NodeIdentifier> nodeIds);
-
-    /**
-     * Returns the set of node identifiers the service is using to retrieve the
-     * flow.
-     *
-     * @return the set of node identifiers the service is using to retrieve the
-     * flow.
-     */
-    Set<NodeIdentifier> getNodeIds();
-
-    /**
-     * @return the retrieval delay in seconds
-     */
-    int getRetrievalDelaySeconds();
-
-    /**
-     * Sets the retrieval delay.
-     *
-     * @param delay the retrieval delay in seconds
-     */
-    void setRetrievalDelay(String delay);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.java
deleted file mode 100644
index b3afc6e..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.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.cluster.flow;
-
-/**
- * Represents the various state of a flow managed by the cluster.
- *
- * The semantics of the values are:
- * <ul>
- * <li> CURRENT - the flow is current </li>
- * <li> STALE - the flow is not current, but is eligible to be updated. </li>
- * <li> UNKNOWN - the flow is not current and is not eligible to be updated.
- * </li>
- * </ul>
- *
- * @author unattributed
- */
-public enum PersistedFlowState {
-
-    CURRENT,
-    STALE,
-    UNKNOWN
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/StaleFlowException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/StaleFlowException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/StaleFlowException.java
deleted file mode 100644
index ce5a08b..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/StaleFlowException.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.cluster.flow;
-
-/**
- * Represents the exceptional case when a caller is requesting the current flow,
- * but a current flow is not available.
- *
- * @author unattributed
- */
-public class StaleFlowException extends RuntimeException {
-
-    public StaleFlowException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public StaleFlowException(String message) {
-        super(message);
-    }
-
-    public StaleFlowException(Throwable cause) {
-        super(cause);
-    }
-
-    public StaleFlowException() {
-    }
-
-}


[21/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerStatusEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerStatusEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerStatusEntity.java
deleted file mode 100644
index 94d8aca..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerStatusEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.status.ControllerStatusDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a ControllerStatusDTO.
- */
-@XmlRootElement(name = "controllerStatusEntity")
-public class ControllerStatusEntity extends Entity {
-
-    private ControllerStatusDTO controllerStatus;
-
-    /**
-     * The ControllerStatusDTO that is being serialized.
-     *
-     * @return The ControllerStatusDTO object
-     */
-    public ControllerStatusDTO getControllerStatus() {
-        return controllerStatus;
-    }
-
-    public void setControllerStatus(ControllerStatusDTO controllerStatus) {
-        this.controllerStatus = controllerStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/CounterEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/CounterEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/CounterEntity.java
deleted file mode 100644
index 666dc75..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/CounterEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.CounterDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a response from the API. This particular entity holds a reference to a
- * CounterDTO.
- */
-@XmlRootElement(name = "counterEntity")
-public class CounterEntity extends Entity {
-
-    private CounterDTO counter;
-
-    /**
-     * Get the counter.
-     *
-     * @return
-     */
-    public CounterDTO getCounter() {
-        return counter;
-    }
-
-    public void setCounter(CounterDTO counter) {
-        this.counter = counter;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/CountersEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/CountersEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/CountersEntity.java
deleted file mode 100644
index bee7669..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/CountersEntity.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.web.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.CountersDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a response from the API. This particular entity holds a reference to a
- * CountersDTO.
- */
-@XmlRootElement(name = "countersEntity")
-public class CountersEntity extends Entity {
-
-    private CountersDTO counters;
-
-    /**
-     * Get the counters which contains all the counter groups and a generation
-     * date.
-     *
-     * @return
-     */
-    public CountersDTO getCounters() {
-        return counters;
-    }
-
-    public void setCounters(CountersDTO counters) {
-        this.counters = counters;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/Entity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/Entity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/Entity.java
deleted file mode 100644
index 6abcbf3..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/Entity.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.web.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.RevisionDTO;
-
-/**
- * A base type for request/response entities.
- */
-@XmlRootElement(name = "entity")
-public class Entity {
-
-    private RevisionDTO revision;
-
-    /**
-     * A revision for this request/response.
-     *
-     * @return
-     */
-    public RevisionDTO getRevision() {
-        return revision;
-    }
-
-    public void setRevision(RevisionDTO revision) {
-        this.revision = revision;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FlowSnippetEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FlowSnippetEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FlowSnippetEntity.java
deleted file mode 100644
index a15cc51..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FlowSnippetEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.FlowSnippetDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a FlowSnippetDTO.
- */
-@XmlRootElement(name = "flowSnippetEntity")
-public class FlowSnippetEntity extends Entity {
-
-    private FlowSnippetDTO contents;
-
-    /**
-     * The SnippetDTO that is being serialized.
-     *
-     * @return The SnippetDTO object
-     */
-    public FlowSnippetDTO getContents() {
-        return contents;
-    }
-
-    public void setContents(FlowSnippetDTO contents) {
-        this.contents = contents;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FunnelEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FunnelEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FunnelEntity.java
deleted file mode 100644
index 8b43fb8..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FunnelEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.FunnelDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a FunnelDTO.
- */
-@XmlRootElement(name = "funnelEntity")
-public class FunnelEntity extends Entity {
-
-    private FunnelDTO funnel;
-
-    /**
-     * The FunnelDTO that is being serialized.
-     *
-     * @return The FunnelDTO object
-     */
-    public FunnelDTO getFunnel() {
-        return funnel;
-    }
-
-    public void setFunnel(FunnelDTO funnel) {
-        this.funnel = funnel;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FunnelsEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FunnelsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FunnelsEntity.java
deleted file mode 100644
index 2b8cbc8..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FunnelsEntity.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.web.api.entity;
-
-import java.util.Set;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.FunnelDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a response to the API. This particular entity holds a reference to a list of
- * FunnelDTOs.
- */
-@XmlRootElement(name = "funnelsEntity")
-public class FunnelsEntity extends Entity {
-
-    private Set<FunnelDTO> funnels;
-
-    /**
-     * The collection of FunnelDTOs that are being serialized.
-     *
-     * @return
-     */
-    public Set<FunnelDTO> getFunnels() {
-        return funnels;
-    }
-
-    public void setFunnels(Set<FunnelDTO> labels) {
-        this.funnels = labels;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/HistoryEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/HistoryEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/HistoryEntity.java
deleted file mode 100644
index f817cce..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/HistoryEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.action.HistoryDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a HistoryDTO.
- */
-@XmlRootElement(name = "historyEntity")
-public class HistoryEntity extends Entity {
-
-    private HistoryDTO history;
-
-    /**
-     * The HistoryDTO that is being serialized.
-     *
-     * @return The LabelDTO object
-     */
-    public HistoryDTO getHistory() {
-        return history;
-    }
-
-    public void setHistory(HistoryDTO history) {
-        this.history = history;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/InputPortEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/InputPortEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/InputPortEntity.java
deleted file mode 100644
index 9fec9d8..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/InputPortEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.PortDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a response to the API. This particular entity holds a reference to an input
- * PortDTO.
- */
-@XmlRootElement(name = "inputPortEntity")
-public class InputPortEntity extends Entity {
-
-    private PortDTO inputPort;
-
-    /**
-     * The input PortDTO that are being serialized.
-     *
-     * @return
-     */
-    public PortDTO getInputPort() {
-        return inputPort;
-    }
-
-    public void setInputPort(PortDTO inputPort) {
-        this.inputPort = inputPort;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/InputPortsEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/InputPortsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/InputPortsEntity.java
deleted file mode 100644
index 8637596..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/InputPortsEntity.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.web.api.entity;
-
-import java.util.Set;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.PortDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a response to the API. This particular entity holds a reference to a list of
- * input PortDTOs.
- */
-@XmlRootElement(name = "inputPortsEntity")
-public class InputPortsEntity extends Entity {
-
-    private Set<PortDTO> inputPorts;
-
-    /**
-     * The collection of input PortDTOs that are being serialized.
-     *
-     * @return
-     */
-    public Set<PortDTO> getInputPorts() {
-        return inputPorts;
-    }
-
-    public void setInputPorts(Set<PortDTO> inputPorts) {
-        this.inputPorts = inputPorts;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LabelEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LabelEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LabelEntity.java
deleted file mode 100644
index 867d781..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LabelEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.LabelDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a LabelDTO.
- */
-@XmlRootElement(name = "labelEntity")
-public class LabelEntity extends Entity {
-
-    private LabelDTO label;
-
-    /**
-     * The LabelDTO that is being serialized.
-     *
-     * @return The LabelDTO object
-     */
-    public LabelDTO getLabel() {
-        return label;
-    }
-
-    public void setLabel(LabelDTO label) {
-        this.label = label;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LabelsEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LabelsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LabelsEntity.java
deleted file mode 100644
index 063eeaf..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LabelsEntity.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.web.api.entity;
-
-import java.util.Set;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.LabelDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a response to the API. This particular entity holds a reference to a list of
- * LabelDTOs.
- */
-@XmlRootElement(name = "labelsEntity")
-public class LabelsEntity extends Entity {
-
-    private Set<LabelDTO> labels;
-
-    /**
-     * The collection of LabelDTOs that are being serialized.
-     *
-     * @return
-     */
-    public Set<LabelDTO> getLabels() {
-        return labels;
-    }
-
-    public void setLabels(Set<LabelDTO> labels) {
-        this.labels = labels;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LineageEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LineageEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LineageEntity.java
deleted file mode 100644
index 5d443af..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LineageEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.provenance.lineage.LineageDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a LineageDTO.
- */
-@XmlRootElement(name = "lineageEntity")
-public class LineageEntity extends Entity {
-
-    private LineageDTO lineage;
-
-    /**
-     * The LineageDTO that is being serialized.
-     *
-     * @return The LineageDTO object
-     */
-    public LineageDTO getLineage() {
-        return lineage;
-    }
-
-    public void setLineage(LineageDTO lineage) {
-        this.lineage = lineage;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeEntity.java
deleted file mode 100644
index ac5b306..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.NodeDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a NodeDTO.
- */
-@XmlRootElement(name = "nodeEntity")
-public class NodeEntity extends Entity {
-
-    private NodeDTO node;
-
-    /**
-     * The NodeDTO that is being serialized.
-     *
-     * @return The NodeDTO object
-     */
-    public NodeDTO getNode() {
-        return node;
-    }
-
-    public void setNode(NodeDTO node) {
-        this.node = node;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeStatusEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeStatusEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeStatusEntity.java
deleted file mode 100644
index d7471da..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeStatusEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.status.NodeStatusDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a NodeStatusDTO.
- */
-@XmlRootElement(name = "nodeStatusEntity")
-public class NodeStatusEntity extends Entity {
-
-    private NodeStatusDTO nodeStatus;
-
-    /**
-     * The NodeStatusDTO that is being serialized.
-     *
-     * @return The NodeStatusDTO object
-     */
-    public NodeStatusDTO getNodeStatus() {
-        return nodeStatus;
-    }
-
-    public void setNodeStatus(NodeStatusDTO nodeStatus) {
-        this.nodeStatus = nodeStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeSystemDiagnosticsEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeSystemDiagnosticsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeSystemDiagnosticsEntity.java
deleted file mode 100644
index 59eafa1..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeSystemDiagnosticsEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.NodeSystemDiagnosticsDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a NodeSystemDiagnosticsDTO.
- */
-@XmlRootElement(name = "nodeSystemDiagnosticsEntity")
-public class NodeSystemDiagnosticsEntity extends Entity {
-
-    private NodeSystemDiagnosticsDTO nodeSystemDiagnostics;
-
-    /**
-     * The NodeSystemDiagnosticsDTO that is being serialized.
-     *
-     * @return The NodeSystemDiagnosticsDTO object
-     */
-    public NodeSystemDiagnosticsDTO getNodeSystemDiagnostics() {
-        return nodeSystemDiagnostics;
-    }
-
-    public void setNodeSystemDiagnostics(NodeSystemDiagnosticsDTO nodeSystemDiagnostics) {
-        this.nodeSystemDiagnostics = nodeSystemDiagnostics;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/OutputPortEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/OutputPortEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/OutputPortEntity.java
deleted file mode 100644
index 07fdab7..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/OutputPortEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.PortDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a response to the API. This particular entity holds a reference to an output
- * PortDTO.
- */
-@XmlRootElement(name = "outputPortEntity")
-public class OutputPortEntity extends Entity {
-
-    private PortDTO outputPort;
-
-    /**
-     * The output PortDTO that are being serialized.
-     *
-     * @return
-     */
-    public PortDTO getOutputPort() {
-        return outputPort;
-    }
-
-    public void setOutputPort(PortDTO outputPort) {
-        this.outputPort = outputPort;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/OutputPortsEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/OutputPortsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/OutputPortsEntity.java
deleted file mode 100644
index 9fa398d..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/OutputPortsEntity.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.web.api.entity;
-
-import java.util.Set;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.PortDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a response to the API. This particular entity holds a reference to a list of
- * output PortDTOs.
- */
-@XmlRootElement(name = "outputPortsEntity")
-public class OutputPortsEntity extends Entity {
-
-    private Set<PortDTO> outputPorts;
-
-    /**
-     * The collection of output PortDTOs that are being serialized.
-     *
-     * @return
-     */
-    public Set<PortDTO> getOutputPorts() {
-        return outputPorts;
-    }
-
-    public void setOutputPorts(Set<PortDTO> outputPorts) {
-        this.outputPorts = outputPorts;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/PrioritizerTypesEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/PrioritizerTypesEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/PrioritizerTypesEntity.java
deleted file mode 100644
index 2ddddd8..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/PrioritizerTypesEntity.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.web.api.entity;
-
-import java.util.Set;
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.DocumentedTypeDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a response to the API. This particular entity holds a reference to a list of
- * prioritizer types.
- */
-@XmlRootElement(name = "prioritizerTypesEntity")
-public class PrioritizerTypesEntity extends Entity {
-
-    private Set<DocumentedTypeDTO> prioritizerTypes;
-
-    /**
-     * The list of prioritizer types that are being serialized.
-     *
-     * @return
-     */
-    public Set<DocumentedTypeDTO> getPrioritizerTypes() {
-        return prioritizerTypes;
-    }
-
-    public void setPrioritizerTypes(Set<DocumentedTypeDTO> prioritizerTypes) {
-        this.prioritizerTypes = prioritizerTypes;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupEntity.java
deleted file mode 100644
index c677ef1..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.ProcessGroupDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a ProcessGroupDTO.
- */
-@XmlRootElement(name = "processGroupEntity")
-public class ProcessGroupEntity extends Entity {
-
-    private ProcessGroupDTO processGroup;
-
-    /**
-     * The ProcessGroupDTO that is being serialized.
-     *
-     * @return The ControllerDTO object
-     */
-    public ProcessGroupDTO getProcessGroup() {
-        return processGroup;
-    }
-
-    public void setProcessGroup(ProcessGroupDTO controller) {
-        this.processGroup = controller;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupStatusEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupStatusEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupStatusEntity.java
deleted file mode 100644
index f0a6e0f..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupStatusEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a ProcessGroupStatusDTO.
- */
-@XmlRootElement(name = "processGroupStatusEntity")
-public class ProcessGroupStatusEntity extends Entity {
-
-    private ProcessGroupStatusDTO processGroupStatus;
-
-    /**
-     * The ProcessGroupStatusDTO that is being serialized.
-     *
-     * @return The ProcessGroupStatusDTO object
-     */
-    public ProcessGroupStatusDTO getProcessGroupStatus() {
-        return processGroupStatus;
-    }
-
-    public void setProcessGroupStatus(ProcessGroupStatusDTO processGroupStatus) {
-        this.processGroupStatus = processGroupStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupsEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupsEntity.java
deleted file mode 100644
index 1fdf23f..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupsEntity.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.web.api.entity;
-
-import java.util.Set;
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.ProcessGroupDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a set of ProcessGroupDTOs.
- */
-@XmlRootElement(name = "processGroupsEntity")
-public class ProcessGroupsEntity extends Entity {
-
-    private Set<ProcessGroupDTO> processGroups;
-
-    /**
-     * The ProcessGroupDTO that is being serialized.
-     *
-     * @return The ProcessGroupDTOs
-     */
-    public Set<ProcessGroupDTO> getProcessGroups() {
-        return processGroups;
-    }
-
-    public void setProcessGroups(Set<ProcessGroupDTO> processGroups) {
-        this.processGroups = processGroups;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorEntity.java
deleted file mode 100644
index 21ad431..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.ProcessorDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a ProcessorDTO.
- */
-@XmlRootElement(name = "processorEntity")
-public class ProcessorEntity extends Entity {
-
-    private ProcessorDTO processor;
-
-    /**
-     * The ProcessorDTO that is being serialized.
-     *
-     * @return The ProcessorDTO object
-     */
-    public ProcessorDTO getProcessor() {
-        return processor;
-    }
-
-    public void setProcessor(ProcessorDTO processor) {
-        this.processor = processor;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.java
deleted file mode 100644
index 19166f7..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.ProcessorHistoryDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a ProcessorHistoryDTO.
- */
-@XmlRootElement(name = "processorHistoryEntity")
-public class ProcessorHistoryEntity extends Entity {
-
-    private ProcessorHistoryDTO propertyHistory;
-
-    /**
-     * The ProcessorHistoryDTO that is being serialized.
-     *
-     * @return The ProcessorHistoryDTO object
-     */
-    public ProcessorHistoryDTO getProcessorHistory() {
-        return propertyHistory;
-    }
-
-    public void setProcessorHistory(ProcessorHistoryDTO propertyHistory) {
-        this.propertyHistory = propertyHistory;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorTypesEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorTypesEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorTypesEntity.java
deleted file mode 100644
index 23237fe..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorTypesEntity.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.web.api.entity;
-
-import java.util.Set;
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.DocumentedTypeDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a response to the API. This particular entity holds a reference to a list of
- * processor types.
- */
-@XmlRootElement(name = "processorTypesEntity")
-public class ProcessorTypesEntity extends Entity {
-
-    private Set<DocumentedTypeDTO> processorTypes;
-
-    /**
-     * The list of processor types that are being serialized.
-     *
-     * @return
-     */
-    public Set<DocumentedTypeDTO> getProcessorTypes() {
-        return processorTypes;
-    }
-
-    public void setProcessorTypes(Set<DocumentedTypeDTO> processorTypes) {
-        this.processorTypes = processorTypes;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorsEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorsEntity.java
deleted file mode 100644
index 4962b38..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorsEntity.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.web.api.entity;
-
-import java.util.Set;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.ProcessorDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a response to the API. This particular entity holds a reference to a list of
- * ProcessorDTOs.
- */
-@XmlRootElement(name = "processorsEntity")
-public class ProcessorsEntity extends Entity {
-
-    private Set<ProcessorDTO> processors;
-
-    /**
-     * The collection of ProcessorDTOs that are being serialized.
-     *
-     * @return
-     */
-    public Set<ProcessorDTO> getProcessors() {
-        return processors;
-    }
-
-    public void setProcessors(Set<ProcessorDTO> processors) {
-        this.processors = processors;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEntity.java
deleted file mode 100644
index 72a8528..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEntity.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.web.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.provenance.ProvenanceDTO;
-
-@XmlRootElement(name = "provenanceEntity")
-public class ProvenanceEntity extends Entity {
-
-    private ProvenanceDTO provenance;
-
-    /**
-     * The provenance .
-     *
-     * @return
-     */
-    public ProvenanceDTO getProvenance() {
-        return provenance;
-    }
-
-    public void setProvenance(ProvenanceDTO provenance) {
-        this.provenance = provenance;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEventEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEventEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEventEntity.java
deleted file mode 100644
index 0aa0a55..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEventEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a ProvenanceEventDTO.
- */
-@XmlRootElement(name = "provenanceEventEntity")
-public class ProvenanceEventEntity extends Entity {
-
-    private ProvenanceEventDTO provenanceEvent;
-
-    /**
-     * The ProvenanceEventDTO that is being serialized.
-     *
-     * @return The ProvenanceEventDTO object
-     */
-    public ProvenanceEventDTO getProvenanceEvent() {
-        return provenanceEvent;
-    }
-
-    public void setProvenanceEvent(ProvenanceEventDTO provenanceEvent) {
-        this.provenanceEvent = provenanceEvent;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceOptionsEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceOptionsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceOptionsEntity.java
deleted file mode 100644
index 01591cb..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceOptionsEntity.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.web.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-
-import org.apache.nifi.web.api.dto.provenance.ProvenanceOptionsDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a ProvenanceOptionsDTO.
- */
-@XmlRootElement(name = "provenanceOptionsEntity")
-public class ProvenanceOptionsEntity extends Entity {
-
-    private ProvenanceOptionsDTO provenanceOptions;
-
-    /**
-     * The ProvenanceOptionsDTO that is being serialized.
-     *
-     * @return The ProvenanceOptionsDTO object
-     */
-    public ProvenanceOptionsDTO getProvenanceOptions() {
-        return provenanceOptions;
-    }
-
-    public void setProvenanceOptions(ProvenanceOptionsDTO provenanceOptions) {
-        this.provenanceOptions = provenanceOptions;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupEntity.java
deleted file mode 100644
index 62a84d6..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a RemoteProcessGroupDTO.
- */
-@XmlRootElement(name = "remoteProcessGroupEntity")
-public class RemoteProcessGroupEntity extends Entity {
-
-    private RemoteProcessGroupDTO remoteProcessGroup;
-
-    /**
-     * The RemoteProcessGroupDTO that is being serialized.
-     *
-     * @return The RemoteProcessGroupDTO object
-     */
-    public RemoteProcessGroupDTO getRemoteProcessGroup() {
-        return remoteProcessGroup;
-    }
-
-    public void setRemoteProcessGroup(RemoteProcessGroupDTO remoteProcessGroup) {
-        this.remoteProcessGroup = remoteProcessGroup;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupPortEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupPortEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupPortEntity.java
deleted file mode 100644
index f310b5e..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupPortEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a RemoteProcessGroupPortDTO.
- */
-@XmlRootElement(name = "remoteProcessGroupPortEntity")
-public class RemoteProcessGroupPortEntity extends Entity {
-
-    private RemoteProcessGroupPortDTO remoteProcessGroupPort;
-
-    /**
-     * The RemoteProcessGroupPortDTO that is being serialized.
-     *
-     * @return The RemoteProcessGroupPortDTO object
-     */
-    public RemoteProcessGroupPortDTO getRemoteProcessGroupPort() {
-        return remoteProcessGroupPort;
-    }
-
-    public void setRemoteProcessGroupPort(RemoteProcessGroupPortDTO remoteProcessGroupPort) {
-        this.remoteProcessGroupPort = remoteProcessGroupPort;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupsEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupsEntity.java
deleted file mode 100644
index a04c789..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupsEntity.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.web.api.entity;
-
-import java.util.Set;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a response to the API. This particular entity holds a reference to a list of
- * RemoteProcessGroupDTOs.
- */
-@XmlRootElement(name = "remoteProcessGroupsEntity")
-public class RemoteProcessGroupsEntity extends Entity {
-
-    private Set<RemoteProcessGroupDTO> remoteProcessGroups;
-
-    /**
-     * The collection of RemoteProcessGroupDTOs that are being serialized.
-     *
-     * @return
-     */
-    public Set<RemoteProcessGroupDTO> getRemoteProcessGroups() {
-        return remoteProcessGroups;
-    }
-
-    public void setRemoteProcessGroups(Set<RemoteProcessGroupDTO> remoteProcessGroups) {
-        this.remoteProcessGroups = remoteProcessGroups;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SearchResultsEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SearchResultsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SearchResultsEntity.java
deleted file mode 100644
index 6b9a88b..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SearchResultsEntity.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.web.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to ProcessorSearchResultDTOs, RemoteProcessGroupSearchResultDTOs,
- * and ConnectionSearchResultDTOs.
- */
-@XmlRootElement(name = "searchResultsEntity")
-public class SearchResultsEntity {
-
-    private SearchResultsDTO searchResultsDTO;
-
-    /**
-     * The search results.
-     *
-     * @return
-     */
-    public SearchResultsDTO getSearchResultsDTO() {
-        return searchResultsDTO;
-    }
-
-    public void setSearchResultsDTO(SearchResultsDTO searchResultsDTO) {
-        this.searchResultsDTO = searchResultsDTO;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SnippetEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SnippetEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SnippetEntity.java
deleted file mode 100644
index a9b05a1..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SnippetEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.SnippetDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a SnippetDTO.
- */
-@XmlRootElement(name = "snippetEntity")
-public class SnippetEntity extends Entity {
-
-    private SnippetDTO snippet;
-
-    /**
-     * The SnippetDTO that is being serialized.
-     *
-     * @return The SnippetDTO object
-     */
-    public SnippetDTO getSnippet() {
-        return snippet;
-    }
-
-    public void setSnippet(SnippetDTO snippet) {
-        this.snippet = snippet;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/StatusHistoryEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/StatusHistoryEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/StatusHistoryEntity.java
deleted file mode 100644
index d3c2247..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/StatusHistoryEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a StatusHistoryDTO.
- */
-@XmlRootElement(name = "statusHistoryEntity")
-public class StatusHistoryEntity extends Entity {
-
-    private StatusHistoryDTO statusHistory;
-
-    /**
-     * The StatusHistoryDTO that is being serialized.
-     *
-     * @return The StatusHistoryDTO object
-     */
-    public StatusHistoryDTO getStatusHistory() {
-        return statusHistory;
-    }
-
-    public void setStatusHistory(StatusHistoryDTO statusHistory) {
-        this.statusHistory = statusHistory;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SystemDiagnosticsEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SystemDiagnosticsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SystemDiagnosticsEntity.java
deleted file mode 100644
index e7baa16..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SystemDiagnosticsEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.SystemDiagnosticsDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a SystemDiagnosticsDTO.
- */
-@XmlRootElement(name = "systemDiagnosticsEntity")
-public class SystemDiagnosticsEntity extends Entity {
-
-    private SystemDiagnosticsDTO systemDiagnostics;
-
-    /**
-     * The SystemDiagnosticsDTO that is being serialized.
-     *
-     * @return The SystemDiagnosticsDTO object
-     */
-    public SystemDiagnosticsDTO getSystemDiagnostics() {
-        return systemDiagnostics;
-    }
-
-    public void setSystemDiagnostics(SystemDiagnosticsDTO health) {
-        this.systemDiagnostics = health;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/TemplateEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/TemplateEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/TemplateEntity.java
deleted file mode 100644
index 8e9f07a..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/TemplateEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.TemplateDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a TemplateDTO.
- */
-@XmlRootElement(name = "templateEntity")
-public class TemplateEntity extends Entity {
-
-    private TemplateDTO template;
-
-    /**
-     * The TemplateDTO that is being serialized.
-     *
-     * @return The TemplateDTO object
-     */
-    public TemplateDTO getTemplate() {
-        return template;
-    }
-
-    public void setTemplate(TemplateDTO template) {
-        this.template = template;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/TemplatesEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/TemplatesEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/TemplatesEntity.java
deleted file mode 100644
index 3400045..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/TemplatesEntity.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.web.api.entity;
-
-import java.util.Date;
-import java.util.Set;
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.TemplateDTO;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a set of TemplateDTOs.
- */
-@XmlRootElement(name = "templatesEntity")
-public class TemplatesEntity extends Entity {
-
-    private Set<TemplateDTO> templates;
-    private Date generated;
-
-    /**
-     * The set of TemplateDTOs that is being serialized.
-     *
-     * @return The TemplateDTO object
-     */
-    public Set<TemplateDTO> getTemplates() {
-        return templates;
-    }
-
-    public void setTemplates(Set<TemplateDTO> templates) {
-        this.templates = templates;
-    }
-
-    /**
-     * When this content was generated.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    public Date getGenerated() {
-        return generated;
-    }
-
-    public void setGenerated(Date generated) {
-        this.generated = generated;
-    }
-}


[42/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
----------------------------------------------------------------------
diff --git a/commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
deleted file mode 100644
index 10e348d..0000000
--- a/commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
+++ /dev/null
@@ -1,876 +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 org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedInputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.InputStream;
-import java.net.InetSocketAddress;
-import java.nio.file.InvalidPathException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-
-public class NiFiProperties extends Properties {
-
-    private static final long serialVersionUID = 2119177359005492702L;
-
-    private static final Logger LOG = LoggerFactory.getLogger(NiFiProperties.class);
-    private static NiFiProperties instance = null;
-
-    // core properties
-    public static final String PROPERTIES_FILE_PATH = "nifi.properties.file.path";
-    public static final String FLOW_CONFIGURATION_FILE = "nifi.flow.configuration.file";
-    public static final String FLOW_CONFIGURATION_ARCHIVE_FILE = "nifi.flow.configuration.archive.file";
-    public static final String TASK_CONFIGURATION_FILE = "nifi.reporting.task.configuration.file";
-    public static final String SERVICE_CONFIGURATION_FILE = "nifi.controller.service.configuration.file";
-    public static final String AUTHORITY_PROVIDER_CONFIGURATION_FILE = "nifi.authority.provider.configuration.file";
-    public static final String REPOSITORY_DATABASE_DIRECTORY = "nifi.database.directory";
-    public static final String RESTORE_DIRECTORY = "nifi.restore.directory";
-    public static final String VERSION = "nifi.version";
-    public static final String WRITE_DELAY_INTERVAL = "nifi.flowservice.writedelay.interval";
-    public static final String AUTO_RESUME_STATE = "nifi.flowcontroller.autoResumeState";
-    public static final String FLOW_CONTROLLER_GRACEFUL_SHUTDOWN_PERIOD = "nifi.flowcontroller.graceful.shutdown.period";
-    public static final String NAR_LIBRARY_DIRECTORY = "nifi.nar.library.directory";
-    public static final String NAR_WORKING_DIRECTORY = "nifi.nar.working.directory";
-    public static final String COMPONENT_DOCS_DIRECTORY = "nifi.documentation.working.directory";
-    public static final String SENSITIVE_PROPS_KEY = "nifi.sensitive.props.key";
-    public static final String SENSITIVE_PROPS_ALGORITHM = "nifi.sensitive.props.algorithm";
-    public static final String SENSITIVE_PROPS_PROVIDER = "nifi.sensitive.props.provider";
-    public static final String H2_URL_APPEND = "nifi.h2.url.append";
-    public static final String REMOTE_INPUT_PORT = "nifi.remote.input.socket.port";
-    public static final String SITE_TO_SITE_SECURE = "nifi.remote.input.secure";
-    public static final String TEMPLATE_DIRECTORY = "nifi.templates.directory";
-    public static final String ADMINISTRATIVE_YIELD_DURATION = "nifi.administrative.yield.duration";
-    public static final String PERSISTENT_STATE_DIRECTORY = "nifi.persistent.state.directory";
-
-    // content repository properties
-    public static final String REPOSITORY_CONTENT_PREFIX = "nifi.content.repository.directory.";
-    public static final String CONTENT_REPOSITORY_IMPLEMENTATION = "nifi.content.repository.implementation";
-    public static final String MAX_APPENDABLE_CLAIM_SIZE = "nifi.content.claim.max.appendable.size";
-    public static final String MAX_FLOWFILES_PER_CLAIM = "nifi.content.claim.max.flow.files";
-    public static final String CONTENT_ARCHIVE_MAX_RETENTION_PERIOD = "nifi.content.repository.archive.max.retention.period";
-    public static final String CONTENT_ARCHIVE_MAX_USAGE_PERCENTAGE = "nifi.content.repository.archive.max.usage.percentage";
-    public static final String CONTENT_ARCHIVE_BACK_PRESSURE_PERCENTAGE = "nifi.content.repository.archive.backpressure.percentage";
-    public static final String CONTENT_ARCHIVE_ENABLED = "nifi.content.repository.archive.enabled";
-    public static final String CONTENT_ARCHIVE_CLEANUP_FREQUENCY = "nifi.content.repository.archive.cleanup.frequency";
-    public static final String CONTENT_VIEWER_URL = "nifi.content.viewer.url";
-
-    // flowfile repository properties
-    public static final String FLOWFILE_REPOSITORY_IMPLEMENTATION = "nifi.flowfile.repository.implementation";
-    public static final String FLOWFILE_REPOSITORY_ALWAYS_SYNC = "nifi.flowfile.repository.always.sync";
-    public static final String FLOWFILE_REPOSITORY_DIRECTORY = "nifi.flowfile.repository.directory";
-    public static final String FLOWFILE_REPOSITORY_PARTITIONS = "nifi.flowfile.repository.partitions";
-    public static final String FLOWFILE_REPOSITORY_CHECKPOINT_INTERVAL = "nifi.flowfile.repository.checkpoint.interval";
-    public static final String FLOWFILE_SWAP_MANAGER_IMPLEMENTATION = "nifi.swap.manager.implementation";
-    public static final String QUEUE_SWAP_THRESHOLD = "nifi.queue.swap.threshold";
-    public static final String SWAP_IN_THREADS = "nifi.swap.in.threads";
-    public static final String SWAP_IN_PERIOD = "nifi.swap.in.period";
-    public static final String SWAP_OUT_THREADS = "nifi.swap.out.threads";
-    public static final String SWAP_OUT_PERIOD = "nifi.swap.out.period";
-
-    // provenance properties
-    public static final String PROVENANCE_REPO_IMPLEMENTATION_CLASS = "nifi.provenance.repository.implementation";
-    public static final String PROVENANCE_REPO_DIRECTORY_PREFIX = "nifi.provenance.repository.directory.";
-    public static final String PROVENANCE_MAX_STORAGE_TIME = "nifi.provenance.repository.max.storage.time";
-    public static final String PROVENANCE_MAX_STORAGE_SIZE = "nifi.provenance.repository.max.storage.size";
-    public static final String PROVENANCE_ROLLOVER_TIME = "nifi.provenance.repository.rollover.time";
-    public static final String PROVENANCE_ROLLOVER_SIZE = "nifi.provenance.repository.rollover.size";
-    public static final String PROVENANCE_QUERY_THREAD_POOL_SIZE = "nifi.provenance.repository.query.threads";
-    public static final String PROVENANCE_COMPRESS_ON_ROLLOVER = "nifi.provenance.repository.compress.on.rollover";
-    public static final String PROVENANCE_INDEXED_FIELDS = "nifi.provenance.repository.indexed.fields";
-    public static final String PROVENANCE_INDEXED_ATTRIBUTES = "nifi.provenance.repository.indexed.attributes";
-    public static final String PROVENANCE_INDEX_SHARD_SIZE = "nifi.provenance.repository.index.shard.size";
-    public static final String PROVENANCE_JOURNAL_COUNT = "nifi.provenance.repository.journal.count";
-
-    // component status repository properties
-    public static final String COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION = "nifi.components.status.repository.implementation";
-    public static final String COMPONENT_STATUS_SNAPSHOT_FREQUENCY = "nifi.components.status.snapshot.frequency";
-
-    // encryptor properties
-    public static final String NF_SENSITIVE_PROPS_KEY = "nifi.sensitive.props.key";
-    public static final String NF_SENSITIVE_PROPS_ALGORITHM = "nifi.sensitive.props.algorithm";
-    public static final String NF_SENSITIVE_PROPS_PROVIDER = "nifi.sensitive.props.provider";
-
-    // security properties
-    public static final String SECURITY_KEYSTORE = "nifi.security.keystore";
-    public static final String SECURITY_KEYSTORE_TYPE = "nifi.security.keystoreType";
-    public static final String SECURITY_KEYSTORE_PASSWD = "nifi.security.keystorePasswd";
-    public static final String SECURITY_KEY_PASSWD = "nifi.security.keyPasswd";
-    public static final String SECURITY_TRUSTSTORE = "nifi.security.truststore";
-    public static final String SECURITY_TRUSTSTORE_TYPE = "nifi.security.truststoreType";
-    public static final String SECURITY_TRUSTSTORE_PASSWD = "nifi.security.truststorePasswd";
-    public static final String SECURITY_NEED_CLIENT_AUTH = "nifi.security.needClientAuth";
-    public static final String SECURITY_USER_AUTHORITY_PROVIDER = "nifi.security.user.authority.provider";
-    public static final String SECURITY_CLUSTER_AUTHORITY_PROVIDER_PORT = "nifi.security.cluster.authority.provider.port";
-    public static final String SECURITY_CLUSTER_AUTHORITY_PROVIDER_THREADS = "nifi.security.cluster.authority.provider.threads";
-    public static final String SECURITY_USER_CREDENTIAL_CACHE_DURATION = "nifi.security.user.credential.cache.duration";
-    public static final String SECURITY_SUPPORT_NEW_ACCOUNT_REQUESTS = "nifi.security.support.new.account.requests";
-    public static final String SECURITY_DEFAULT_USER_ROLES = "nifi.security.default.user.roles";
-    public static final String SECURITY_OCSP_RESPONDER_URL = "nifi.security.ocsp.responder.url";
-    public static final String SECURITY_OCSP_RESPONDER_CERTIFICATE = "nifi.security.ocsp.responder.certificate";
-
-    // web properties
-    public static final String WEB_WAR_DIR = "nifi.web.war.directory";
-    public static final String WEB_HTTP_PORT = "nifi.web.http.port";
-    public static final String WEB_HTTP_HOST = "nifi.web.http.host";
-    public static final String WEB_HTTPS_PORT = "nifi.web.https.port";
-    public static final String WEB_HTTPS_HOST = "nifi.web.https.host";
-    public static final String WEB_WORKING_DIR = "nifi.web.jetty.working.directory";
-    public static final String WEB_THREADS = "nifi.web.jetty.threads";
-
-    // ui properties
-    public static final String UI_BANNER_TEXT = "nifi.ui.banner.text";
-    public static final String UI_AUTO_REFRESH_INTERVAL = "nifi.ui.autorefresh.interval";
-
-    // cluster common properties
-    public static final String CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL = "nifi.cluster.protocol.heartbeat.interval";
-    public static final String CLUSTER_PROTOCOL_IS_SECURE = "nifi.cluster.protocol.is.secure";
-    public static final String CLUSTER_PROTOCOL_SOCKET_TIMEOUT = "nifi.cluster.protocol.socket.timeout";
-    public static final String CLUSTER_PROTOCOL_CONNECTION_HANDSHAKE_TIMEOUT = "nifi.cluster.protocol.connection.handshake.timeout";
-    public static final String CLUSTER_PROTOCOL_USE_MULTICAST = "nifi.cluster.protocol.use.multicast";
-    public static final String CLUSTER_PROTOCOL_MULTICAST_ADDRESS = "nifi.cluster.protocol.multicast.address";
-    public static final String CLUSTER_PROTOCOL_MULTICAST_PORT = "nifi.cluster.protocol.multicast.port";
-    public static final String CLUSTER_PROTOCOL_MULTICAST_SERVICE_BROADCAST_DELAY = "nifi.cluster.protocol.multicast.service.broadcast.delay";
-    public static final String CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS = "nifi.cluster.protocol.multicast.service.locator.attempts";
-    public static final String CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS_DELAY = "nifi.cluster.protocol.multicast.service.locator.attempts.delay";
-
-    // cluster node properties
-    public static final String CLUSTER_IS_NODE = "nifi.cluster.is.node";
-    public static final String CLUSTER_NODE_ADDRESS = "nifi.cluster.node.address";
-    public static final String CLUSTER_NODE_PROTOCOL_PORT = "nifi.cluster.node.protocol.port";
-    public static final String CLUSTER_NODE_PROTOCOL_THREADS = "nifi.cluster.node.protocol.threads";
-    public static final String CLUSTER_NODE_UNICAST_MANAGER_ADDRESS = "nifi.cluster.node.unicast.manager.address";
-    public static final String CLUSTER_NODE_UNICAST_MANAGER_PROTOCOL_PORT = "nifi.cluster.node.unicast.manager.protocol.port";
-
-    // cluster manager properties
-    public static final String CLUSTER_IS_MANAGER = "nifi.cluster.is.manager";
-    public static final String CLUSTER_MANAGER_ADDRESS = "nifi.cluster.manager.address";
-    public static final String CLUSTER_MANAGER_PROTOCOL_PORT = "nifi.cluster.manager.protocol.port";
-    public static final String CLUSTER_MANAGER_NODE_FIREWALL_FILE = "nifi.cluster.manager.node.firewall.file";
-    public static final String CLUSTER_MANAGER_NODE_EVENT_HISTORY_SIZE = "nifi.cluster.manager.node.event.history.size";
-    public static final String CLUSTER_MANAGER_NODE_API_CONNECTION_TIMEOUT = "nifi.cluster.manager.node.api.connection.timeout";
-    public static final String CLUSTER_MANAGER_NODE_API_READ_TIMEOUT = "nifi.cluster.manager.node.api.read.timeout";
-    public static final String CLUSTER_MANAGER_NODE_API_REQUEST_THREADS = "nifi.cluster.manager.node.api.request.threads";
-    public static final String CLUSTER_MANAGER_FLOW_RETRIEVAL_DELAY = "nifi.cluster.manager.flow.retrieval.delay";
-    public static final String CLUSTER_MANAGER_PROTOCOL_THREADS = "nifi.cluster.manager.protocol.threads";
-    public static final String CLUSTER_MANAGER_SAFEMODE_DURATION = "nifi.cluster.manager.safemode.duration";
-
-    // defaults
-    public static final String DEFAULT_TITLE = "NiFi";
-    public static final Boolean DEFAULT_AUTO_RESUME_STATE = true;
-    public static final String DEFAULT_AUTHORITY_PROVIDER_CONFIGURATION_FILE = "conf/authority-providers.xml";
-    public static final String DEFAULT_USER_CREDENTIAL_CACHE_DURATION = "24 hours";
-    public static final Integer DEFAULT_REMOTE_INPUT_PORT = null;
-    public static final Path DEFAULT_TEMPLATE_DIRECTORY = Paths.get("conf", "templates");
-    public static final int DEFAULT_WEB_THREADS = 200;
-    public static final String DEFAULT_WEB_WORKING_DIR = "./work/jetty";
-    public static final String DEFAULT_NAR_WORKING_DIR = "./work/nar";
-    public static final String DEFAULT_COMPONENT_DOCS_DIRECTORY = "./work/docs/components";
-    public static final String DEFAULT_NAR_LIBRARY_DIR = "./lib";
-    public static final String DEFAULT_FLOWFILE_REPO_PARTITIONS = "256";
-    public static final String DEFAULT_FLOWFILE_CHECKPOINT_INTERVAL = "2 min";
-    public static final int DEFAULT_MAX_FLOWFILES_PER_CLAIM = 100;
-    public static final int DEFAULT_QUEUE_SWAP_THRESHOLD = 20000;
-    public static final String DEFAULT_SWAP_STORAGE_LOCATION = "./flowfile_repository/swap";
-    public static final String DEFAULT_SWAP_IN_PERIOD = "1 sec";
-    public static final String DEFAULT_SWAP_OUT_PERIOD = "5 sec";
-    public static final int DEFAULT_SWAP_IN_THREADS = 4;
-    public static final int DEFAULT_SWAP_OUT_THREADS = 4;
-    public static final String DEFAULT_ADMINISTRATIVE_YIELD_DURATION = "30 sec";
-    public static final String DEFAULT_PERSISTENT_STATE_DIRECTORY = "./conf/state";
-    public static final String DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY = "5 mins";
-
-    // cluster common defaults
-    public static final String DEFAULT_CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL = "5 sec";
-    public static final String DEFAULT_CLUSTER_PROTOCOL_MULTICAST_SERVICE_BROADCAST_DELAY = "500 ms";
-    public static final int DEFAULT_CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS = 3;
-    public static final String DEFAULT_CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS_DELAY = "1 sec";
-    public static final String DEFAULT_CLUSTER_PROTOCOL_SOCKET_TIMEOUT = "30 sec";
-    public static final String DEFAULT_CLUSTER_PROTOCOL_CONNECTION_HANDSHAKE_TIMEOUT = "45 sec";
-
-    // cluster node defaults
-    public static final int DEFAULT_CLUSTER_NODE_PROTOCOL_THREADS = 2;
-
-    // cluster manager defaults
-    public static final int DEFAULT_CLUSTER_MANAGER_NODE_EVENT_HISTORY_SIZE = 10;
-    public static final String DEFAULT_CLUSTER_MANAGER_NODE_API_CONNECTION_TIMEOUT = "30 sec";
-    public static final String DEFAULT_CLUSTER_MANAGER_NODE_API_READ_TIMEOUT = "30 sec";
-    public static final int DEFAULT_CLUSTER_MANAGER_NODE_API_NUM_REQUEST_THREADS = 10;
-    public static final String DEFAULT_CLUSTER_MANAGER_FLOW_RETRIEVAL_DELAY = "5 sec";
-    public static final int DEFAULT_CLUSTER_MANAGER_PROTOCOL_THREADS = 10;
-    public static final String DEFAULT_CLUSTER_MANAGER_SAFEMODE_DURATION = "0 sec";
-
-    private NiFiProperties() {
-        super();
-    }
-
-    /**
-     * This is the method through which the NiFiProperties object should be
-     * obtained.
-     *
-     * @return the NiFiProperties object to use
-     * @throws RuntimeException if unable to load properties file
-     */
-    public static synchronized NiFiProperties getInstance() {
-        if (null == instance) {
-            final NiFiProperties suspectInstance = new NiFiProperties();
-            final String nfPropertiesFilePath = System.getProperty(NiFiProperties.PROPERTIES_FILE_PATH);
-            if (null == nfPropertiesFilePath || nfPropertiesFilePath.trim().length() == 0) {
-                throw new RuntimeException("Requires a system property called \'" + NiFiProperties.PROPERTIES_FILE_PATH + "\' and this is not set or has no value");
-            }
-            final File propertiesFile = new File(nfPropertiesFilePath);
-            if (!propertiesFile.exists()) {
-                throw new RuntimeException("Properties file doesn't exist \'" + propertiesFile.getAbsolutePath() + "\'");
-            }
-            if (!propertiesFile.canRead()) {
-                throw new RuntimeException("Properties file exists but cannot be read \'" + propertiesFile.getAbsolutePath() + "\'");
-            }
-            InputStream inStream = null;
-            try {
-                inStream = new BufferedInputStream(new FileInputStream(propertiesFile));
-                suspectInstance.load(inStream);
-            } catch (final Exception ex) {
-                LOG.error("Cannot load properties file due to " + ex.getLocalizedMessage());
-                throw new RuntimeException("Cannot load properties file due to " + ex.getLocalizedMessage(), ex);
-            } finally {
-                if (null != inStream) {
-                    try {
-                        inStream.close();
-                    } catch (final Exception ex) {
-                        /**
-                         * do nothing *
-                         */
-                    }
-                }
-            }
-            instance = suspectInstance;
-        }
-        return instance;
-    }
-
-    // getters for core properties //
-    public File getFlowConfigurationFile() {
-        try {
-            return new File(getProperty(FLOW_CONFIGURATION_FILE));
-        } catch (Exception ex) {
-            return null;
-        }
-    }
-
-    public File getFlowConfigurationFileDir() {
-        try {
-            return getFlowConfigurationFile().getParentFile();
-        } catch (Exception ex) {
-            return null;
-        }
-    }
-
-    private Integer getPropertyAsPort(final String propertyName, final Integer defaultValue) {
-        final String port = getProperty(propertyName);
-        if (StringUtils.isEmpty(port)) {
-            return defaultValue;
-        }
-        try {
-            final int val = Integer.parseInt(port);
-            if (val <= 0 || val > 65535) {
-                throw new RuntimeException("Valid port range is 0 - 65535 but got " + val);
-            }
-            return val;
-        } catch (final NumberFormatException e) {
-            return defaultValue;
-        }
-    }
-
-    public int getQueueSwapThreshold() {
-        final String thresholdValue = getProperty(QUEUE_SWAP_THRESHOLD);
-        if (thresholdValue == null) {
-            return DEFAULT_QUEUE_SWAP_THRESHOLD;
-        }
-
-        try {
-            return Integer.parseInt(thresholdValue);
-        } catch (final NumberFormatException e) {
-            return DEFAULT_QUEUE_SWAP_THRESHOLD;
-        }
-    }
-
-    public Integer getIntegerProperty(final String propertyName, final Integer defaultValue) {
-        final String value = getProperty(propertyName);
-        if (value == null) {
-            return defaultValue;
-        }
-
-        try {
-            return Integer.parseInt(getProperty(propertyName));
-        } catch (final Exception e) {
-            return defaultValue;
-        }
-    }
-
-    public int getSwapInThreads() {
-        return getIntegerProperty(SWAP_IN_THREADS, DEFAULT_SWAP_IN_THREADS);
-    }
-
-    public int getSwapOutThreads() {
-        final String value = getProperty(SWAP_OUT_THREADS);
-        if (value == null) {
-            return DEFAULT_SWAP_OUT_THREADS;
-        }
-
-        try {
-            return Integer.parseInt(getProperty(SWAP_OUT_THREADS));
-        } catch (final Exception e) {
-            return DEFAULT_SWAP_OUT_THREADS;
-        }
-    }
-
-    public String getSwapInPeriod() {
-        return getProperty(SWAP_IN_PERIOD, DEFAULT_SWAP_IN_PERIOD);
-    }
-
-    public String getSwapOutPeriod() {
-        return getProperty(SWAP_OUT_PERIOD, DEFAULT_SWAP_OUT_PERIOD);
-    }
-
-    public String getAdministrativeYieldDuration() {
-        return getProperty(ADMINISTRATIVE_YIELD_DURATION, DEFAULT_ADMINISTRATIVE_YIELD_DURATION);
-    }
-
-    /**
-     * The socket port to listen on for a Remote Input Port.
-     *
-     * @return
-     */
-    public Integer getRemoteInputPort() {
-        return getPropertyAsPort(REMOTE_INPUT_PORT, DEFAULT_REMOTE_INPUT_PORT);
-    }
-
-    /**
-     * @return False if property value is 'false'; True otherwise.
-     */
-    public Boolean isSiteToSiteSecure() {
-        final String secureVal = getProperty(SITE_TO_SITE_SECURE, "true");
-
-        if ("false".equalsIgnoreCase(secureVal)) {
-            return false;
-        }else{
-            return true;
-        }
-
-    }
-
-    /**
-     * Returns the directory to which Templates are to be persisted
-     *
-     * @return
-     */
-    public Path getTemplateDirectory() {
-        final String strVal = getProperty(TEMPLATE_DIRECTORY);
-        return (strVal == null) ? DEFAULT_TEMPLATE_DIRECTORY : Paths.get(strVal);
-    }
-
-    /**
-     * Get the flow service write delay.
-     *
-     * @return The write delay
-     */
-    public String getFlowServiceWriteDelay() {
-        return getProperty(WRITE_DELAY_INTERVAL);
-    }
-
-    /**
-     * Returns whether the processors should be started automatically when the
-     * application loads.
-     *
-     * @return Whether to auto start the processors or not
-     */
-    public boolean getAutoResumeState() {
-        final String rawAutoResumeState = getProperty(AUTO_RESUME_STATE, DEFAULT_AUTO_RESUME_STATE.toString());
-        return Boolean.parseBoolean(rawAutoResumeState);
-    }
-
-    /**
-     * Returns the number of partitions that should be used for the FlowFile
-     * Repository
-     *
-     * @return
-     */
-    public int getFlowFileRepositoryPartitions() {
-        final String rawProperty = getProperty(FLOWFILE_REPOSITORY_PARTITIONS, DEFAULT_FLOWFILE_REPO_PARTITIONS);
-        return Integer.parseInt(rawProperty);
-    }
-
-    /**
-     * Returns the number of milliseconds between FlowFileRepository
-     * checkpointing
-     *
-     * @return
-     */
-    public String getFlowFileRepositoryCheckpointInterval() {
-        return getProperty(FLOWFILE_REPOSITORY_CHECKPOINT_INTERVAL, DEFAULT_FLOWFILE_CHECKPOINT_INTERVAL);
-    }
-
-    /**
-     * @return the restore directory or null if not configured
-     */
-    public File getRestoreDirectory() {
-        final String value = getProperty(RESTORE_DIRECTORY);
-        if (StringUtils.isBlank(value)) {
-            return null;
-        } else {
-            return new File(value);
-        }
-    }
-
-    /**
-     * @return the user authorities file
-     */
-    public File getAuthorityProviderConfiguraitonFile() {
-        final String value = getProperty(AUTHORITY_PROVIDER_CONFIGURATION_FILE);
-        if (StringUtils.isBlank(value)) {
-            return new File(DEFAULT_AUTHORITY_PROVIDER_CONFIGURATION_FILE);
-        } else {
-            return new File(value);
-        }
-    }
-
-    /**
-     * Will default to true unless the value is explicitly set to false.
-     *
-     * @return Whether client auth is required
-     */
-    public boolean getNeedClientAuth() {
-        boolean needClientAuth = true;
-        String rawNeedClientAuth = getProperty(SECURITY_NEED_CLIENT_AUTH);
-        if ("false".equalsIgnoreCase(rawNeedClientAuth)) {
-            needClientAuth = false;
-        }
-        return needClientAuth;
-    }
-
-    public String getUserCredentialCacheDuration() {
-        return getProperty(SECURITY_USER_CREDENTIAL_CACHE_DURATION, DEFAULT_USER_CREDENTIAL_CACHE_DURATION);
-    }
-
-    public boolean getSupportNewAccountRequests() {
-        boolean shouldSupport = true;
-        String rawShouldSupport = getProperty(SECURITY_SUPPORT_NEW_ACCOUNT_REQUESTS);
-        if ("false".equalsIgnoreCase(rawShouldSupport)) {
-            shouldSupport = false;
-        }
-        return shouldSupport;
-    }
-
-    // getters for web properties //
-    public Integer getPort() {
-        Integer port = null;
-        try {
-            port = Integer.parseInt(getProperty(WEB_HTTP_PORT));
-        } catch (NumberFormatException nfe) {
-        }
-        return port;
-    }
-
-    public Integer getSslPort() {
-        Integer sslPort = null;
-        try {
-            sslPort = Integer.parseInt(getProperty(WEB_HTTPS_PORT));
-        } catch (NumberFormatException nfe) {
-        }
-        return sslPort;
-    }
-
-    public int getWebThreads() {
-        return getIntegerProperty(WEB_THREADS, DEFAULT_WEB_THREADS);
-    }
-    
-    public File getWebWorkingDirectory() {
-        return new File(getProperty(WEB_WORKING_DIR, DEFAULT_WEB_WORKING_DIR));
-    }
-
-    public File getComponentDocumentationWorkingDirectory() {
-        return new File(getProperty(COMPONENT_DOCS_DIRECTORY, DEFAULT_COMPONENT_DOCS_DIRECTORY));
-    }
-
-    public File getNarWorkingDirectory() {
-        return new File(getProperty(NAR_WORKING_DIRECTORY, DEFAULT_NAR_WORKING_DIR));
-    }
-
-    public File getFrameworkWorkingDirectory() {
-        return new File(getNarWorkingDirectory(), "framework");
-    }
-
-    public File getExtensionsWorkingDirectory() {
-        return new File(getNarWorkingDirectory(), "extensions");
-    }
-
-    public File getNarLibraryDirectory() {
-        return new File(getProperty(NAR_LIBRARY_DIRECTORY, DEFAULT_NAR_LIBRARY_DIR));
-    }
-
-    // getters for ui properties //
-    /**
-     * Get the title for the UI.
-     *
-     * @return The UI title
-     */
-    public String getUiTitle() {
-        return this.getProperty(VERSION, DEFAULT_TITLE);
-    }
-
-    /**
-     * Get the banner text.
-     *
-     * @return The banner text
-     */
-    public String getBannerText() {
-        return this.getProperty(UI_BANNER_TEXT, StringUtils.EMPTY);
-    }
-
-    /**
-     * Returns the auto refresh interval in seconds.
-     *
-     * @return
-     */
-    public String getAutoRefreshInterval() {
-        return getProperty(UI_AUTO_REFRESH_INTERVAL);
-    }
-
-    // getters for cluster protocol properties //
-    public String getClusterProtocolHeartbeatInterval() {
-        return getProperty(CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL, DEFAULT_CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL);
-    }
-
-    public String getNodeHeartbeatInterval() {
-        return getClusterProtocolHeartbeatInterval();
-    }
-
-    public String getClusterProtocolSocketTimeout() {
-        return getProperty(CLUSTER_PROTOCOL_SOCKET_TIMEOUT, DEFAULT_CLUSTER_PROTOCOL_SOCKET_TIMEOUT);
-    }
-
-    public String getClusterProtocolConnectionHandshakeTimeout() {
-        return getProperty(CLUSTER_PROTOCOL_CONNECTION_HANDSHAKE_TIMEOUT, DEFAULT_CLUSTER_PROTOCOL_CONNECTION_HANDSHAKE_TIMEOUT);
-    }
-
-    public boolean getClusterProtocolUseMulticast() {
-        return Boolean.parseBoolean(getProperty(CLUSTER_PROTOCOL_USE_MULTICAST));
-    }
-
-    public InetSocketAddress getClusterProtocolMulticastAddress() {
-        try {
-            String multicastAddress = getProperty(CLUSTER_PROTOCOL_MULTICAST_ADDRESS);
-            int multicastPort = Integer.parseInt(getProperty(CLUSTER_PROTOCOL_MULTICAST_PORT));
-            return new InetSocketAddress(multicastAddress, multicastPort);
-        } catch (Exception ex) {
-            throw new RuntimeException("Invalid multicast address/port due to: " + ex, ex);
-        }
-    }
-
-    public String getClusterProtocolMulticastServiceBroadcastDelay() {
-        return getProperty(CLUSTER_PROTOCOL_MULTICAST_SERVICE_BROADCAST_DELAY);
-    }
-
-    public File getPersistentStateDirectory() {
-        final String dirName = getProperty(PERSISTENT_STATE_DIRECTORY, DEFAULT_PERSISTENT_STATE_DIRECTORY);
-        final File file = new File(dirName);
-        if (!file.exists()) {
-            file.mkdirs();
-        }
-        return file;
-    }
-
-    public int getClusterProtocolMulticastServiceLocatorAttempts() {
-        try {
-            return Integer.parseInt(getProperty(CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS));
-        } catch (NumberFormatException nfe) {
-            return DEFAULT_CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS;
-        }
-    }
-
-    public String getClusterProtocolMulticastServiceLocatorAttemptsDelay() {
-        return getProperty(CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS_DELAY, DEFAULT_CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS_DELAY);
-    }
-
-    // getters for cluster node properties //
-    public boolean isNode() {
-        return Boolean.parseBoolean(getProperty(CLUSTER_IS_NODE));
-    }
-
-    public InetSocketAddress getClusterNodeProtocolAddress() {
-        try {
-            String socketAddress = getProperty(CLUSTER_NODE_ADDRESS);
-            if (StringUtils.isBlank(socketAddress)) {
-                socketAddress = "localhost";
-            }
-            int socketPort = getClusterNodeProtocolPort();
-            return InetSocketAddress.createUnresolved(socketAddress, socketPort);
-        } catch (Exception ex) {
-            throw new RuntimeException("Invalid node protocol address/port due to: " + ex, ex);
-        }
-    }
-
-    public Integer getClusterNodeProtocolPort() {
-        try {
-            return Integer.parseInt(getProperty(CLUSTER_NODE_PROTOCOL_PORT));
-        } catch (NumberFormatException nfe) {
-            return null;
-        }
-    }
-
-    public int getClusterNodeProtocolThreads() {
-        try {
-            return Integer.parseInt(getProperty(CLUSTER_NODE_PROTOCOL_THREADS));
-        } catch (NumberFormatException nfe) {
-            return DEFAULT_CLUSTER_NODE_PROTOCOL_THREADS;
-        }
-    }
-
-    public InetSocketAddress getClusterNodeUnicastManagerProtocolAddress() {
-        try {
-            String socketAddress = getProperty(CLUSTER_NODE_UNICAST_MANAGER_ADDRESS);
-            if (StringUtils.isBlank(socketAddress)) {
-                socketAddress = "localhost";
-            }
-            int socketPort = Integer.parseInt(getProperty(CLUSTER_NODE_UNICAST_MANAGER_PROTOCOL_PORT));
-            return InetSocketAddress.createUnresolved(socketAddress, socketPort);
-        } catch (Exception ex) {
-            throw new RuntimeException("Invalid unicast manager address/port due to: " + ex, ex);
-        }
-    }
-
-    // getters for cluster manager properties //
-    public boolean isClusterManager() {
-        return Boolean.parseBoolean(getProperty(CLUSTER_IS_MANAGER));
-    }
-
-    public InetSocketAddress getClusterManagerProtocolAddress() {
-        try {
-            String socketAddress = getProperty(CLUSTER_MANAGER_ADDRESS);
-            if (StringUtils.isBlank(socketAddress)) {
-                socketAddress = "localhost";
-            }
-            int socketPort = getClusterManagerProtocolPort();
-            return InetSocketAddress.createUnresolved(socketAddress, socketPort);
-        } catch (Exception ex) {
-            throw new RuntimeException("Invalid manager protocol address/port due to: " + ex, ex);
-        }
-    }
-
-    public Integer getClusterManagerProtocolPort() {
-        try {
-            return Integer.parseInt(getProperty(CLUSTER_MANAGER_PROTOCOL_PORT));
-        } catch (NumberFormatException nfe) {
-            return null;
-        }
-    }
-
-    public File getClusterManagerNodeFirewallFile() {
-        final String firewallFile = getProperty(CLUSTER_MANAGER_NODE_FIREWALL_FILE);
-        if (StringUtils.isBlank(firewallFile)) {
-            return null;
-        } else {
-            return new File(firewallFile);
-        }
-    }
-
-    public int getClusterManagerNodeEventHistorySize() {
-        try {
-            return Integer.parseInt(getProperty(CLUSTER_MANAGER_NODE_EVENT_HISTORY_SIZE));
-        } catch (NumberFormatException nfe) {
-            return DEFAULT_CLUSTER_MANAGER_NODE_EVENT_HISTORY_SIZE;
-        }
-    }
-
-    public String getClusterManagerNodeApiConnectionTimeout() {
-        return getProperty(CLUSTER_MANAGER_NODE_API_CONNECTION_TIMEOUT, DEFAULT_CLUSTER_MANAGER_NODE_API_CONNECTION_TIMEOUT);
-    }
-
-    public String getClusterManagerNodeApiReadTimeout() {
-        return getProperty(CLUSTER_MANAGER_NODE_API_READ_TIMEOUT, DEFAULT_CLUSTER_MANAGER_NODE_API_READ_TIMEOUT);
-    }
-
-    public int getClusterManagerNodeApiRequestThreads() {
-        try {
-            return Integer.parseInt(getProperty(CLUSTER_MANAGER_NODE_API_REQUEST_THREADS));
-        } catch (NumberFormatException nfe) {
-            return DEFAULT_CLUSTER_MANAGER_NODE_API_NUM_REQUEST_THREADS;
-        }
-    }
-
-    public String getClusterManagerFlowRetrievalDelay() {
-        return getProperty(CLUSTER_MANAGER_FLOW_RETRIEVAL_DELAY, DEFAULT_CLUSTER_MANAGER_FLOW_RETRIEVAL_DELAY);
-    }
-
-    public int getClusterManagerProtocolThreads() {
-        try {
-            return Integer.parseInt(getProperty(CLUSTER_MANAGER_PROTOCOL_THREADS));
-        } catch (NumberFormatException nfe) {
-            return DEFAULT_CLUSTER_MANAGER_PROTOCOL_THREADS;
-        }
-    }
-
-    public String getClusterManagerSafeModeDuration() {
-        return getProperty(CLUSTER_MANAGER_SAFEMODE_DURATION, DEFAULT_CLUSTER_MANAGER_SAFEMODE_DURATION);
-    }
-
-    public String getClusterProtocolManagerToNodeApiScheme() {
-        final String isSecureProperty = getProperty(CLUSTER_PROTOCOL_IS_SECURE);
-        if (Boolean.valueOf(isSecureProperty)) {
-            return "https";
-        } else {
-            return "http";
-        }
-    }
-
-    public InetSocketAddress getNodeApiAddress() {
-
-        final String rawScheme = getClusterProtocolManagerToNodeApiScheme();
-        final String scheme = (rawScheme == null) ? "http" : rawScheme;
-
-        final String host;
-        final int port;
-        if ("http".equalsIgnoreCase(scheme)) {
-            // get host
-            if (StringUtils.isBlank(getProperty(WEB_HTTP_HOST))) {
-                host = "localhost";
-            } else {
-                host = getProperty(WEB_HTTP_HOST);
-            }
-            // get port
-            port = getPort();
-        } else {
-            // get host
-            if (StringUtils.isBlank(getProperty(WEB_HTTPS_HOST))) {
-                host = "localhost";
-            } else {
-                host = getProperty(WEB_HTTPS_HOST);
-            }
-            // get port
-            port = getSslPort();
-        }
-
-        return InetSocketAddress.createUnresolved(host, port);
-
-    }
-
-    /**
-     * Returns the database repository path. It simply returns the value
-     * configured. No directories will be created as a result of this operation.
-     *
-     * @return database repository path
-     * @throws InvalidPathException If the configured path is invalid
-     */
-    public Path getDatabaseRepositoryPath() {
-        return Paths.get(getProperty(REPOSITORY_DATABASE_DIRECTORY));
-    }
-
-    /**
-     * Returns the flow file repository path. It simply returns the value
-     * configured. No directories will be created as a result of this operation.
-     *
-     * @return database repository path
-     * @throws InvalidPathException If the configured path is invalid
-     */
-    public Path getFlowFileRepositoryPath() {
-        return Paths.get(getProperty(FLOWFILE_REPOSITORY_DIRECTORY));
-    }
-
-    /**
-     * Returns the content repository paths. This method returns a mapping of
-     * file repository name to file repository paths. It simply returns the
-     * values configured. No directories will be created as a result of this
-     * operation.
-     *
-     * @return file repositories paths
-     * @throws InvalidPathException If any of the configured paths are invalid
-     */
-    public Map<String, Path> getContentRepositoryPaths() {
-        final Map<String, Path> contentRepositoryPaths = new HashMap<>();
-
-        // go through each property
-        for (String propertyName : stringPropertyNames()) {
-            // determine if the property is a file repository path
-            if (StringUtils.startsWith(propertyName, REPOSITORY_CONTENT_PREFIX)) {
-                // get the repository key
-                final String key = StringUtils.substringAfter(propertyName, REPOSITORY_CONTENT_PREFIX);
-
-                // attempt to resolve the path specified
-                contentRepositoryPaths.put(key, Paths.get(getProperty(propertyName)));
-            }
-        }
-        return contentRepositoryPaths;
-    }
-
-    /**
-     * Returns the provenance repository paths. This method returns a mapping of
-     * file repository name to file repository paths. It simply returns the
-     * values configured. No directories will be created as a result of this
-     * operation.
-     *
-     * @return
-     */
-    public Map<String, Path> getProvenanceRepositoryPaths() {
-        final Map<String, Path> provenanceRepositoryPaths = new HashMap<>();
-
-        // go through each property
-        for (String propertyName : stringPropertyNames()) {
-            // determine if the property is a file repository path
-            if (StringUtils.startsWith(propertyName, PROVENANCE_REPO_DIRECTORY_PREFIX)) {
-                // get the repository key
-                final String key = StringUtils.substringAfter(propertyName, PROVENANCE_REPO_DIRECTORY_PREFIX);
-
-                // attempt to resolve the path specified
-                provenanceRepositoryPaths.put(key, Paths.get(getProperty(propertyName)));
-            }
-        }
-        return provenanceRepositoryPaths;
-    }
-
-    public int getMaxFlowFilesPerClaim() {
-        try {
-            return Integer.parseInt(getProperty(MAX_FLOWFILES_PER_CLAIM));
-        } catch (NumberFormatException nfe) {
-            return DEFAULT_MAX_FLOWFILES_PER_CLAIM;
-        }
-    }
-
-    public String getMaxAppendableClaimSize() {
-        return getProperty(MAX_APPENDABLE_CLAIM_SIZE);
-    }
-
-    @Override
-    public String getProperty(final String key, final String defaultValue) {
-        final String value = super.getProperty(key, defaultValue);
-        if (value == null) {
-            return null;
-        }
-
-        if (value.trim().isEmpty()) {
-            return defaultValue;
-        }
-        return value;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-properties/src/main/java/org/apache/nifi/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/commons/nifi-properties/src/main/java/org/apache/nifi/util/StringUtils.java b/commons/nifi-properties/src/main/java/org/apache/nifi/util/StringUtils.java
deleted file mode 100644
index aa6f8f3..0000000
--- a/commons/nifi-properties/src/main/java/org/apache/nifi/util/StringUtils.java
+++ /dev/null
@@ -1,66 +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;
-
-/**
- * String Utils based on the Apache Commons Lang String Utils.
- * These simple util methods here allow us to avoid a dependency in the core
- */
-public class StringUtils {
-
-    public static final String EMPTY = "";
-
-    public static boolean isBlank(final String str) {
-        if (str == null || str.isEmpty()) {
-            return true;
-        }
-        for (int i = 0; i < str.length(); i++) {
-            if (!Character.isWhitespace(str.charAt(i))) {
-                return false;
-            }
-        }
-        return true;
-    }
-
-    public static boolean isEmpty(final String str) {
-        return str == null || str.isEmpty();
-    }
-
-    public static boolean startsWith(final String str, final String prefix) {
-        if (str == null || prefix == null) {
-            return (str == null && prefix == null);
-        }
-        if (prefix.length() > str.length()) {
-            return false;
-        }
-        return str.regionMatches(false, 0, prefix, 0, prefix.length());
-    }
-
-    public static String substringAfter(final String str, final String separator) {
-        if (isEmpty(str)) {
-            return str;
-        }
-        if (separator == null) {
-            return EMPTY;
-        }
-        int pos = str.indexOf(separator);
-        if (pos == -1) {
-            return EMPTY;
-        }
-        return str.substring(pos + separator.length());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-security-utils/pom.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-security-utils/pom.xml b/commons/nifi-security-utils/pom.xml
deleted file mode 100644
index 0eaaeb4..0000000
--- a/commons/nifi-security-utils/pom.xml
+++ /dev/null
@@ -1,40 +0,0 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-    <!--
-      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.
-    -->
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-commons-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-
-    <artifactId>nifi-security-utils</artifactId>
-    <version>0.0.1-SNAPSHOT</version>
-    <name>NiFi Security Utils</name>
-    <description>Contains security functionality.</description>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>slf4j-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-lang3</artifactId>
-        </dependency>
-    </dependencies>
-</project>
-

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/CertificateUtils.java
----------------------------------------------------------------------
diff --git a/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/CertificateUtils.java b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/CertificateUtils.java
deleted file mode 100644
index 087d891..0000000
--- a/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/CertificateUtils.java
+++ /dev/null
@@ -1,158 +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.security.util;
-
-import java.io.BufferedInputStream;
-import java.io.IOException;
-import java.net.URL;
-import java.security.KeyStore;
-import java.security.cert.CertificateParsingException;
-import java.security.cert.X509Certificate;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public final class CertificateUtils {
-
-    private static final Logger logger = LoggerFactory.getLogger(CertificateUtils.class);
-
-    /**
-     * Returns true if the given keystore can be loaded using the given keystore
-     * type and password. Returns false otherwise.
-     * @param keystore
-     * @param keystoreType
-     * @param password
-     * @return 
-     */
-    public static boolean isStoreValid(final URL keystore, final KeystoreType keystoreType, final char[] password) {
-
-        if (keystore == null) {
-            throw new IllegalArgumentException("keystore may not be null");
-        } else if (keystoreType == null) {
-            throw new IllegalArgumentException("keystore type may not be null");
-        } else if (password == null) {
-            throw new IllegalArgumentException("password may not be null");
-        }
-
-        BufferedInputStream bis = null;
-        final KeyStore ks;
-        try {
-
-            // load the keystore
-            bis = new BufferedInputStream(keystore.openStream());
-            ks = KeyStore.getInstance(keystoreType.name());
-            ks.load(bis, password);
-
-            return true;
-
-        } catch (Exception e) {
-            return false;
-        } finally {
-            if (bis != null) {
-                try {
-                    bis.close();
-                } catch (final IOException ioe) {
-                    logger.warn("Failed to close input stream", ioe);
-                }
-            }
-        }
-    }
-
-    /**
-     * Extracts the username from the specified DN. If the username cannot be
-     * extracted because the CN is in an unrecognized format, the entire CN is
-     * returned. If the CN cannot be extracted because the DN is in an
-     * unrecognized format, the entire DN is returned.
-     *
-     * @param dn
-     * @return
-     */
-    public static String extractUsername(String dn) {
-        String username = dn;
-        String cn = "";
-
-        // ensure the dn is specified
-        if (StringUtils.isNotBlank(dn)) {
-
-            // attempt to locate the cn
-            if (dn.startsWith("CN=")) {
-                cn = StringUtils.substringBetween(dn, "CN=", ",");
-            } else if (dn.startsWith("/CN=")) {
-                cn = StringUtils.substringBetween(dn, "CN=", "/");
-            } else if (dn.startsWith("C=") || dn.startsWith("/C=")) {
-                cn = StringUtils.substringAfter(dn, "CN=");
-            } else if (dn.startsWith("/") && StringUtils.contains(dn, "CN=")) {
-                cn = StringUtils.substringAfter(dn, "CN=");
-            }
-
-            // attempt to get the username from the cn
-            if (StringUtils.isNotBlank(cn)) {
-                if (cn.endsWith(")")) {
-                    username = StringUtils.substringBetween(cn, "(", ")");
-                } else if (cn.contains(" ")) {
-                    username = StringUtils.substringAfterLast(cn, " ");
-                } else {
-                    username = cn;
-                }
-            }
-        }
-
-        return username;
-    }
-
-    /**
-     * Returns a list of subject alternative names. Any name that is represented
-     * as a String by X509Certificate.getSubjectAlternativeNames() is converted
-     * to lowercase and returned.
-     *
-     * @param certificate a certificate
-     * @return a list of subject alternative names; list is never null
-     * @throws CertificateParsingException if parsing the certificate failed
-     */
-    public static List<String> getSubjectAlternativeNames(final X509Certificate certificate) throws CertificateParsingException {
-
-        final Collection<List<?>> altNames = certificate.getSubjectAlternativeNames();
-        if (altNames == null) {
-            return new ArrayList<>();
-        }
-
-        final List<String> result = new ArrayList<>();
-        for (final List<?> generalName : altNames) {
-            /* 
-             * generalName has the name type as the first element a String or 
-             * byte array for the second element.  We return any general names 
-             * that are String types.
-             * 
-             * We don't inspect the numeric name type because some certificates
-             * incorrectly put IPs and DNS names under the wrong name types.
-             */
-            final Object value = generalName.get(1);
-            if (value instanceof String) {
-                result.add(((String) value).toLowerCase());
-            }
-
-        }
-
-        return result;
-    }
-
-    private CertificateUtils() {
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/EncryptionMethod.java
----------------------------------------------------------------------
diff --git a/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/EncryptionMethod.java b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/EncryptionMethod.java
deleted file mode 100644
index 741fdde..0000000
--- a/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/EncryptionMethod.java
+++ /dev/null
@@ -1,84 +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.security.util;
-
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-/**
- * Enumeration capturing essential information about the various encryption
- * methods that might be supported.
- *
- * @author none
- */
-public enum EncryptionMethod {
-
-    MD5_128AES("PBEWITHMD5AND128BITAES-CBC-OPENSSL", "BC", false),
-    MD5_256AES("PBEWITHMD5AND256BITAES-CBC-OPENSSL", "BC", false),
-    SHA1_RC2("PBEWITHSHA1ANDRC2", "BC", false),
-    SHA1_DES("PBEWITHSHA1ANDDES", "BC", false),
-    MD5_192AES("PBEWITHMD5AND192BITAES-CBC-OPENSSL", "BC", false),
-    MD5_DES("PBEWITHMD5ANDDES", "BC", false),
-    MD5_RC2("PBEWITHMD5ANDRC2", "BC", false),
-    SHA_192AES("PBEWITHSHAAND192BITAES-CBC-BC", "BC", true),
-    SHA_40RC4("PBEWITHSHAAND40BITRC4", "BC", true),
-    SHA256_128AES("PBEWITHSHA256AND128BITAES-CBC-BC", "BC", true),
-    SHA_128RC2("PBEWITHSHAAND128BITRC2-CBC", "BC", true),
-    SHA_128AES("PBEWITHSHAAND128BITAES-CBC-BC", "BC", true),
-    SHA256_192AES("PBEWITHSHA256AND192BITAES-CBC-BC", "BC", true),
-    SHA_2KEYTRIPLEDES("PBEWITHSHAAND2-KEYTRIPLEDES-CBC", "BC", true),
-    SHA256_256AES("PBEWITHSHA256AND256BITAES-CBC-BC", "BC", true),
-    SHA_40RC2("PBEWITHSHAAND40BITRC2-CBC", "BC", true),
-    SHA_256AES("PBEWITHSHAAND256BITAES-CBC-BC", "BC", true),
-    SHA_3KEYTRIPLEDES("PBEWITHSHAAND3-KEYTRIPLEDES-CBC", "BC", true),
-    SHA_TWOFISH("PBEWITHSHAANDTWOFISH-CBC", "BC", true),
-    SHA_128RC4("PBEWITHSHAAND128BITRC4", "BC", true);
-    private final String algorithm;
-    private final String provider;
-    private final boolean unlimitedStrength;
-
-    EncryptionMethod(String algorithm, String provider, boolean unlimitedStrength) {
-        this.algorithm = algorithm;
-        this.provider = provider;
-        this.unlimitedStrength = unlimitedStrength;
-    }
-
-    public String getProvider() {
-        return provider;
-    }
-
-    public String getAlgorithm() {
-        return algorithm;
-    }
-
-    /**
-     * @return true if algorithm requires unlimited strength policies
-     */
-    public boolean isUnlimitedStrength() {
-        return unlimitedStrength;
-    }
-
-    @Override
-    public String toString() {
-        final ToStringBuilder builder = new ToStringBuilder(this);
-        ToStringBuilder.setDefaultStyle(ToStringStyle.SHORT_PREFIX_STYLE);
-        builder.append("algorithm name", algorithm);
-        builder.append("Requires unlimited strength JCE policy", unlimitedStrength);
-        builder.append("Algorithm Provider", provider);
-        return builder.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/KeystoreType.java
----------------------------------------------------------------------
diff --git a/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/KeystoreType.java b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/KeystoreType.java
deleted file mode 100644
index 18574bb..0000000
--- a/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/KeystoreType.java
+++ /dev/null
@@ -1,26 +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.security.util;
-
-/**
- * Keystore types.
- */
-public enum KeystoreType {
-
-    PKCS12,
-    JKS;
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SecurityStoreTypes.java
----------------------------------------------------------------------
diff --git a/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SecurityStoreTypes.java b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SecurityStoreTypes.java
deleted file mode 100644
index 9abfcc3..0000000
--- a/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SecurityStoreTypes.java
+++ /dev/null
@@ -1,144 +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.security.util;
-
-import java.io.PrintWriter;
-import java.io.Writer;
-
-/**
- * Types of security stores and their related Java system properties.
- */
-public enum SecurityStoreTypes {
-
-    TRUSTSTORE(
-            "javax.net.ssl.trustStore",
-            "javax.net.ssl.trustStorePassword",
-            "javax.net.ssl.trustStoreType"),
-    KEYSTORE(
-            "javax.net.ssl.keyStore",
-            "javax.net.ssl.keyStorePassword",
-            "javax.net.ssl.keyStoreType");
-
-    /**
-     * Logs the keystore and truststore Java system property values to the given
-     * writer. It logPasswords is true, then the keystore and truststore
-     * password property values are logged.
-     *
-     * @param writer a writer to log to
-     *
-     * @param logPasswords true if passwords should be logged; false otherwise
-     */
-    public static void logProperties(final Writer writer,
-            final boolean logPasswords) {
-        if (writer == null) {
-            return;
-        }
-
-        PrintWriter pw = new PrintWriter(writer);
-
-        // keystore properties
-        pw.println(
-                KEYSTORE.getStoreProperty() + " = " + System.getProperty(KEYSTORE.getStoreProperty()));
-
-        if (logPasswords) {
-            pw.println(
-                    KEYSTORE.getStorePasswordProperty() + " = "
-                    + System.getProperty(KEYSTORE.getStoreProperty()));
-        }
-
-        pw.println(
-                KEYSTORE.getStoreTypeProperty() + " = "
-                + System.getProperty(KEYSTORE.getStoreTypeProperty()));
-
-        // truststore properties
-        pw.println(
-                TRUSTSTORE.getStoreProperty() + " = "
-                + System.getProperty(TRUSTSTORE.getStoreProperty()));
-
-        if (logPasswords) {
-            pw.println(
-                    TRUSTSTORE.getStorePasswordProperty() + " = "
-                    + System.getProperty(TRUSTSTORE.getStoreProperty()));
-        }
-
-        pw.println(
-                TRUSTSTORE.getStoreTypeProperty() + " = "
-                + System.getProperty(TRUSTSTORE.getStoreTypeProperty()));
-        pw.flush();
-    }
-
-    /**
-     * the Java system property for setting the keystore (or truststore) path
-     */
-    private String storeProperty = "";
-
-    /**
-     * the Java system property for setting the keystore (or truststore)
-     * password
-     */
-    private String storePasswordProperty = "";
-
-    /**
-     * the Java system property for setting the keystore (or truststore) type
-     */
-    private String storeTypeProperty = "";
-
-    /**
-     * Creates an instance.
-     *
-     * @param storeProperty the Java system property for setting the keystore (
-     * or truststore) path
-     * @param storePasswordProperty the Java system property for setting the
-     * keystore (or truststore) password
-     * @param storeTypeProperty the Java system property for setting the
-     * keystore (or truststore) type
-     */
-    SecurityStoreTypes(final String storeProperty,
-            final String storePasswordProperty,
-            final String storeTypeProperty) {
-        this.storeProperty = storeProperty;
-        this.storePasswordProperty = storePasswordProperty;
-        this.storeTypeProperty = storeTypeProperty;
-    }
-
-    /**
-     * Returns the keystore (or truststore) property.
-     *
-     * @return the keystore (or truststore) property
-     */
-    public String getStoreProperty() {
-        return storeProperty;
-    }
-
-    /**
-     * Returns the keystore (or truststore) password property.
-     *
-     * @return the keystore (or truststore) password property
-     */
-    public String getStorePasswordProperty() {
-        return storePasswordProperty;
-    }
-
-    /**
-     * Returns the keystore (or truststore) type property.
-     *
-     * @return the keystore (or truststore) type property
-     */
-    public String getStoreTypeProperty() {
-        return storeTypeProperty;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SslContextFactory.java
----------------------------------------------------------------------
diff --git a/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SslContextFactory.java b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SslContextFactory.java
deleted file mode 100644
index 2371b0c..0000000
--- a/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SslContextFactory.java
+++ /dev/null
@@ -1,180 +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.security.util;
-
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.security.KeyManagementException;
-import java.security.KeyStore;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.SecureRandom;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-
-import javax.net.ssl.KeyManager;
-import javax.net.ssl.KeyManagerFactory;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.TrustManager;
-import javax.net.ssl.TrustManagerFactory;
-
-/**
- * A factory for creating SSL contexts using the application's security
- * properties.
- *
- * @author unattributed
- */
-public final class SslContextFactory {
-
-    public static enum ClientAuth {
-
-        WANT,
-        REQUIRED,
-        NONE
-    }
-
-    /**
-     * Creates a SSLContext instance using the given information.
-     *
-     * @param keystore the full path to the keystore
-     * @param keystorePasswd the keystore password
-     * @param keystoreType the type of keystore (e.g., PKCS12, JKS)
-     * @param truststore the full path to the truststore
-     * @param truststorePasswd the truststore password
-     * @param truststoreType the type of truststore (e.g., PKCS12, JKS)
-     * @param clientAuth the type of client authentication
-     *
-     * @return a SSLContext instance
-     * @throws java.security.KeyStoreException
-     * @throws java.io.IOException
-     * @throws java.security.NoSuchAlgorithmException
-     * @throws java.security.cert.CertificateException
-     * @throws java.security.UnrecoverableKeyException
-     * @throws java.security.KeyManagementException
-     */
-    public static SSLContext createSslContext(
-            final String keystore, final char[] keystorePasswd, final String keystoreType,
-            final String truststore, final char[] truststorePasswd, final String truststoreType,
-            final ClientAuth clientAuth)
-            throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
-            UnrecoverableKeyException, KeyManagementException {
-
-        // prepare the keystore
-        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
-        try (final InputStream keyStoreStream = new FileInputStream(keystore)) {
-            keyStore.load(keyStoreStream, keystorePasswd);
-        }
-        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
-        keyManagerFactory.init(keyStore, keystorePasswd);
-
-        // prepare the truststore
-        final KeyStore trustStore = KeyStore.getInstance(truststoreType);
-        try (final InputStream trustStoreStream = new FileInputStream(truststore)) {
-            trustStore.load(trustStoreStream, truststorePasswd);
-        }
-        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
-        trustManagerFactory.init(trustStore);
-
-        // initialize the ssl context
-        final SSLContext sslContext = SSLContext.getInstance("TLS");
-        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), new SecureRandom());
-        if (ClientAuth.REQUIRED == clientAuth) {
-            sslContext.getDefaultSSLParameters().setNeedClientAuth(true);
-        } else if (ClientAuth.WANT == clientAuth) {
-            sslContext.getDefaultSSLParameters().setWantClientAuth(true);
-        } else {
-            sslContext.getDefaultSSLParameters().setWantClientAuth(false);
-        }
-
-        return sslContext;
-
-    }
-
-    /**
-     * Creates a SSLContext instance using the given information.
-     *
-     * @param keystore the full path to the keystore
-     * @param keystorePasswd the keystore password
-     * @param keystoreType the type of keystore (e.g., PKCS12, JKS)
-     *
-     * @return a SSLContext instance
-     * @throws java.security.KeyStoreException
-     * @throws java.io.IOException
-     * @throws java.security.NoSuchAlgorithmException
-     * @throws java.security.cert.CertificateException
-     * @throws java.security.UnrecoverableKeyException
-     * @throws java.security.KeyManagementException
-     */
-    public static SSLContext createSslContext(
-            final String keystore, final char[] keystorePasswd, final String keystoreType)
-            throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
-            UnrecoverableKeyException, KeyManagementException {
-
-        // prepare the keystore
-        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
-        try (final InputStream keyStoreStream = new FileInputStream(keystore)) {
-            keyStore.load(keyStoreStream, keystorePasswd);
-        }
-        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
-        keyManagerFactory.init(keyStore, keystorePasswd);
-
-        // initialize the ssl context
-        final SSLContext ctx = SSLContext.getInstance("TLS");
-        ctx.init(keyManagerFactory.getKeyManagers(), new TrustManager[0], new SecureRandom());
-
-        return ctx;
-
-    }
-
-    /**
-     * Creates a SSLContext instance using the given information.
-     *
-     * @param truststore the full path to the truststore
-     * @param truststorePasswd the truststore password
-     * @param truststoreType the type of truststore (e.g., PKCS12, JKS)
-     *
-     * @return a SSLContext instance
-     * @throws java.security.KeyStoreException
-     * @throws java.io.IOException
-     * @throws java.security.NoSuchAlgorithmException
-     * @throws java.security.cert.CertificateException
-     * @throws java.security.UnrecoverableKeyException
-     * @throws java.security.KeyManagementException
-     */
-    public static SSLContext createTrustSslContext(
-            final String truststore, final char[] truststorePasswd, final String truststoreType)
-            throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
-            UnrecoverableKeyException, KeyManagementException {
-
-        // prepare the truststore
-        final KeyStore trustStore = KeyStore.getInstance(truststoreType);
-        try (final InputStream trustStoreStream = new FileInputStream(truststore)) {
-            trustStore.load(trustStoreStream, truststorePasswd);
-        }
-        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
-        trustManagerFactory.init(trustStore);
-
-        // initialize the ssl context
-        final SSLContext ctx = SSLContext.getInstance("TLS");
-        ctx.init(new KeyManager[0], trustManagerFactory.getTrustManagers(), new SecureRandom());
-
-        return ctx;
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/pom.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/pom.xml b/commons/nifi-socket-utils/pom.xml
deleted file mode 100644
index efb5a8e..0000000
--- a/commons/nifi-socket-utils/pom.xml
+++ /dev/null
@@ -1,60 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-commons-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-
-    <artifactId>nifi-socket-utils</artifactId>
-    <version>0.0.1-SNAPSHOT</version>
-    <name>NiFi Socket Utils</name>
-    <description>Utilities for socket communication</description>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>slf4j-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-logging-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-net</groupId>
-            <artifactId>commons-net</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-lang3</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-io</groupId>
-            <artifactId>commons-io</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-properties</artifactId>
-        </dependency>
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/AbstractChannelReader.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/AbstractChannelReader.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/AbstractChannelReader.java
deleted file mode 100644
index 172c593..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/AbstractChannelReader.java
+++ /dev/null
@@ -1,166 +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.io.nio;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.SelectionKey;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.nifi.io.nio.consumer.StreamConsumer;
-import org.apache.nifi.io.nio.consumer.StreamConsumerFactory;
-
-import org.apache.commons.lang3.builder.EqualsBuilder;
-import org.apache.commons.lang3.builder.HashCodeBuilder;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- *
- * @author none
- */
-public abstract class AbstractChannelReader implements Runnable {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(AbstractChannelReader.class);
-    private final String uniqueId;
-    private final SelectionKey key;
-    private final BufferPool bufferPool;
-    private final StreamConsumer consumer;
-    private final AtomicBoolean isClosed = new AtomicBoolean(false);
-    private final AtomicReference<ScheduledFuture<?>> future = new AtomicReference<>(null);//the future on which this reader runs...
-
-    public AbstractChannelReader(final String id, final SelectionKey key, final BufferPool empties, final StreamConsumerFactory consumerFactory) {
-        this.uniqueId = id;
-        this.key = key;
-        this.bufferPool = empties;
-        this.consumer = consumerFactory.newInstance(id);
-        consumer.setReturnBufferQueue(bufferPool);
-    }
-
-    protected void setScheduledFuture(final ScheduledFuture<?> future) {
-        this.future.set(future);
-    }
-
-    protected ScheduledFuture<?> getScheduledFuture() {
-        return future.get();
-    }
-
-    protected SelectionKey getSelectionKey() {
-        return key;
-    }
-
-    public boolean isClosed() {
-        return isClosed.get();
-    }
-
-    private void closeStream() {
-        if (isClosed.get()) {
-            return;
-        }
-        try {
-            isClosed.set(true);
-            future.get().cancel(false);
-            key.cancel();
-            key.channel().close();
-        } catch (final IOException ioe) {
-            LOGGER.warn("Unable to cleanly close stream due to " + ioe);
-        } finally {
-            consumer.signalEndOfStream();
-        }
-    }
-
-    /**
-     * Allows a subclass to specifically handle how it reads from the given
-     * key's channel into the given buffer.
-     *
-     * @param key
-     * @param buffer
-     * @return the number of bytes read in the final read cycle. A value of zero
-     * or more indicates the channel is still open but a value of -1 indicates
-     * end of stream.
-     * @throws IOException
-     */
-    protected abstract int fillBuffer(SelectionKey key, ByteBuffer buffer) throws IOException;
-
-    @Override
-    public final void run() {
-        if (!key.isValid() || consumer.isConsumerFinished()) {
-            closeStream();
-            return;
-        }
-        if (!key.isReadable()) {
-            return;//there is nothing available to read...or we aren't allow to read due to throttling
-        }
-        ByteBuffer buffer = null;
-        try {
-            buffer = bufferPool.poll();
-            if (buffer == null) {
-                return; // no buffers available - come back later
-            }
-            final int bytesRead = fillBuffer(key, buffer);
-            buffer.flip();
-            if (buffer.remaining() > 0) {
-                consumer.addFilledBuffer(buffer);
-                buffer = null; //clear the reference - is now the consumer's responsiblity
-            } else {
-                buffer.clear();
-                bufferPool.returnBuffer(buffer, 0);
-                buffer = null; //clear the reference - is now back to the queue
-            }
-            if (bytesRead < 0) { //we've reached the end
-                closeStream();
-            }
-        } catch (final Exception ioe) {
-            closeStream();
-            LOGGER.error("Closed channel reader " + this + " due to " + ioe);
-        } finally {
-            if (buffer != null) {
-                buffer.clear();
-                bufferPool.returnBuffer(buffer, 0);
-            }
-        }
-    }
-
-    @Override
-    public final boolean equals(final Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (obj == this) {
-            return true;
-        }
-        if (obj.getClass() != getClass()) {
-            return false;
-        }
-        AbstractChannelReader rhs = (AbstractChannelReader) obj;
-        return new EqualsBuilder().appendSuper(super.equals(obj)).append(uniqueId, rhs.uniqueId).isEquals();
-    }
-
-    @Override
-    public final int hashCode() {
-        return new HashCodeBuilder(17, 37).append(uniqueId).toHashCode();
-    }
-
-    @Override
-    public final String toString() {
-        return new ToStringBuilder(this, ToStringStyle.NO_FIELD_NAMES_STYLE).append(uniqueId).toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/BufferPool.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/BufferPool.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/BufferPool.java
deleted file mode 100644
index a413ad2..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/BufferPool.java
+++ /dev/null
@@ -1,114 +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.io.nio;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.List;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingDeque;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- *
- * @author none
- */
-public class BufferPool implements Runnable {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(BufferPool.class);
-    final BlockingQueue<ByteBuffer> bufferPool;
-    private final static double ONE_MB = 1 << 20;
-    private Calendar lastRateSampleTime = Calendar.getInstance();
-    private final Calendar startTime = Calendar.getInstance();
-    double lastRateSampleMBps = -1.0;
-    double overallMBps = -1.0;
-    private long totalBytesExtracted = 0L;
-    private long lastTotalBytesExtracted = 0L;
-    final double maxRateMBps;
-
-    public BufferPool(final int bufferCount, final int bufferCapacity, final boolean allocateDirect, final double maxRateMBps) {
-        bufferPool = new LinkedBlockingDeque<>(BufferPool.createBuffers(bufferCount, bufferCapacity, allocateDirect));
-        this.maxRateMBps = maxRateMBps;
-    }
-
-    /**
-     * Returns the given buffer to the pool - and clears it.
-     *
-     * @param buffer
-     * @param bytesProcessed
-     * @return
-     */
-    public synchronized boolean returnBuffer(ByteBuffer buffer, final int bytesProcessed) {
-        totalBytesExtracted += bytesProcessed;
-        buffer.clear();
-        return bufferPool.add(buffer);
-    }
-
-    //here we enforce the desired rate we want by restricting access to buffers when we're over rate
-    public synchronized ByteBuffer poll() {
-        computeRate();
-        final double weightedAvg = (lastRateSampleMBps * 0.7) + (overallMBps * 0.3);
-        if (overallMBps >= maxRateMBps || weightedAvg >= maxRateMBps) {
-            return null;
-        }
-        return bufferPool.poll();
-    }
-
-    public int size() {
-        return bufferPool.size();
-    }
-
-    private synchronized void computeRate() {
-        final Calendar now = Calendar.getInstance();
-        final long measurementDurationMillis = now.getTimeInMillis() - lastRateSampleTime.getTimeInMillis();
-        final double duractionSecs = ((double) measurementDurationMillis) / 1000.0;
-        if (duractionSecs >= 0.75) { //recompute every 3/4 second or when we're too fast
-            final long totalDuractionMillis = now.getTimeInMillis() - startTime.getTimeInMillis();
-            final double totalDurationSecs = ((double) totalDuractionMillis) / 1000.0;
-            final long differenceBytes = totalBytesExtracted - lastTotalBytesExtracted;
-            lastTotalBytesExtracted = totalBytesExtracted;
-            lastRateSampleTime = now;
-            final double bps = ((double) differenceBytes) / duractionSecs;
-            final double totalBps = ((double) totalBytesExtracted / totalDurationSecs);
-            lastRateSampleMBps = bps / ONE_MB;
-            overallMBps = totalBps / ONE_MB;
-        }
-    }
-
-    public static List<ByteBuffer> createBuffers(final int bufferCount, final int bufferCapacity, final boolean allocateDirect) {
-        final List<ByteBuffer> buffers = new ArrayList<>();
-        for (int i = 0; i < bufferCount; i++) {
-            final ByteBuffer buffer = (allocateDirect) ? ByteBuffer.allocateDirect(bufferCapacity) : ByteBuffer.allocate(bufferCapacity);
-            buffers.add(buffer);
-        }
-        return buffers;
-    }
-
-    private void logChannelReadRates() {
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug(String.format("Overall rate= %,.4f MB/s / Current Rate= %,.4f MB/s / Total Bytes Read= %d", overallMBps, lastRateSampleMBps, totalBytesExtracted));
-        }
-    }
-
-    @Override
-    public void run() {
-        computeRate();
-        logChannelReadRates();
-    }
-}


[46/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/BooleanCastEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/BooleanCastEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/BooleanCastEvaluator.java
deleted file mode 100644
index a88b72b..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/BooleanCastEvaluator.java
+++ /dev/null
@@ -1,50 +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.attribute.expression.language.evaluation.cast;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-
-public class BooleanCastEvaluator extends BooleanEvaluator {
-
-    private final StringEvaluator subjectEvaluator;
-
-    public BooleanCastEvaluator(final StringEvaluator subjectEvaluator) {
-        this.subjectEvaluator = subjectEvaluator;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final QueryResult<String> result = subjectEvaluator.evaluate(attributes);
-        if (result.getValue() == null) {
-            return new BooleanQueryResult(null);
-        }
-
-        return new BooleanQueryResult(Boolean.valueOf(result.getValue().trim()));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subjectEvaluator;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/DateCastEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/DateCastEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/DateCastEvaluator.java
deleted file mode 100644
index 96e74ef..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/DateCastEvaluator.java
+++ /dev/null
@@ -1,117 +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.attribute.expression.language.evaluation.cast;
-
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Locale;
-import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.DateQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
-import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-
-public class DateCastEvaluator extends DateEvaluator {
-
-    public static final String DATE_TO_STRING_FORMAT = "EEE MMM dd HH:mm:ss zzz yyyy";
-    public static final Pattern DATE_TO_STRING_PATTERN = Pattern.compile("(?:[a-zA-Z]{3} ){2}\\d{2} \\d{2}\\:\\d{2}\\:\\d{2} (?:.*?) \\d{4}");
-
-    public static final String ALTERNATE_FORMAT_WITHOUT_MILLIS = "yyyy/MM/dd HH:mm:ss";
-    public static final String ALTERNATE_FORMAT_WITH_MILLIS = "yyyy/MM/dd HH:mm:ss.SSS";
-    public static final Pattern ALTERNATE_PATTERN = Pattern.compile("\\d{4}/\\d{2}/\\d{2} \\d{2}\\:\\d{2}\\:\\d{2}(\\.\\d{3})?");
-
-    public static final Pattern NUMBER_PATTERN = Pattern.compile("\\d+");
-
-    private final Evaluator<?> subjectEvaluator;
-
-    public DateCastEvaluator(final Evaluator<?> subjectEvaluator) {
-        if (subjectEvaluator.getResultType() == ResultType.BOOLEAN) {
-            throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + subjectEvaluator.getResultType() + " to " + ResultType.DATE);
-        }
-
-        this.subjectEvaluator = subjectEvaluator;
-    }
-
-    @Override
-    public QueryResult<Date> evaluate(final Map<String, String> attributes) {
-        final QueryResult<?> result = subjectEvaluator.evaluate(attributes);
-        if (result.getValue() == null) {
-            return new DateQueryResult(null);
-        }
-
-        switch (result.getResultType()) {
-            case DATE:
-                return (DateQueryResult) result;
-            case STRING:
-                final String value = ((StringQueryResult) result).getValue().trim();
-                if (DATE_TO_STRING_PATTERN.matcher(value).matches()) {
-                    final SimpleDateFormat sdf = new SimpleDateFormat(DATE_TO_STRING_FORMAT, Locale.US);
-
-                    try {
-                        final Date date = sdf.parse(value);
-                        return new DateQueryResult(date);
-                    } catch (final ParseException pe) {
-                        final String details = "Format: '" + DATE_TO_STRING_FORMAT + "' Value: '" + value + "'";
-                        throw new AttributeExpressionLanguageException("Could not parse date using " + details, pe);
-                    }
-                } else if (NUMBER_PATTERN.matcher(value).matches()) {
-                    return new DateQueryResult(new Date(Long.valueOf(value)));
-                } else {
-                    final Matcher altMatcher = ALTERNATE_PATTERN.matcher(value);
-                    if (altMatcher.matches()) {
-                        final String millisValue = altMatcher.group(1);
-
-                        final String format;
-                        if (millisValue == null) {
-                            format = ALTERNATE_FORMAT_WITHOUT_MILLIS;
-                        } else {
-                            format = ALTERNATE_FORMAT_WITH_MILLIS;
-                        }
-
-                        final SimpleDateFormat sdf = new SimpleDateFormat(format, Locale.US);
-
-                        try {
-                            final Date date = sdf.parse(value);
-                            return new DateQueryResult(date);
-                        } catch (final ParseException pe) {
-                            throw new AttributeExpressionLanguageException("Could not parse input as date", pe);
-                        }
-                    } else {
-                        throw new AttributeExpressionLanguageException("Could not implicitly convert input to DATE: " + value);
-                    }
-                }
-            case NUMBER:
-                return new DateQueryResult(new Date((Long) result.getValue()));
-            default:
-                return new DateQueryResult(null);
-        }
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subjectEvaluator;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/NumberCastEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/NumberCastEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/NumberCastEvaluator.java
deleted file mode 100644
index 0447d48..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/NumberCastEvaluator.java
+++ /dev/null
@@ -1,72 +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.attribute.expression.language.evaluation.cast;
-
-import java.util.Map;
-import java.util.regex.Pattern;
-
-import org.apache.nifi.attribute.expression.language.evaluation.DateQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-
-public class NumberCastEvaluator extends NumberEvaluator {
-
-    private final Evaluator<?> subjectEvaluator;
-    private static final Pattern NUMBER_PATTERN = Pattern.compile("-?\\d+");
-
-    public NumberCastEvaluator(final Evaluator<?> subjectEvaluator) {
-        if (subjectEvaluator.getResultType() == ResultType.BOOLEAN) {
-            throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + subjectEvaluator.getResultType() + " to " + ResultType.NUMBER);
-        }
-        this.subjectEvaluator = subjectEvaluator;
-    }
-
-    @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final QueryResult<?> result = subjectEvaluator.evaluate(attributes);
-        if (result.getValue() == null) {
-            return new NumberQueryResult(null);
-        }
-
-        switch (result.getResultType()) {
-            case NUMBER:
-                return (NumberQueryResult) result;
-            case STRING:
-                final String trimmed = ((StringQueryResult) result).getValue().trim();
-                if (NUMBER_PATTERN.matcher(trimmed).matches()) {
-                    return new NumberQueryResult(Long.valueOf(trimmed));
-                } else {
-                    return new NumberQueryResult(null);
-                }
-            case DATE:
-                return new NumberQueryResult(((DateQueryResult) result).getValue().getTime());
-            default:
-                return new NumberQueryResult(null);
-        }
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subjectEvaluator;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/StringCastEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/StringCastEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/StringCastEvaluator.java
deleted file mode 100644
index 1a3732a..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/StringCastEvaluator.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.attribute.expression.language.evaluation.cast;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class StringCastEvaluator extends StringEvaluator {
-
-    private final Evaluator<?> subjectEvaluator;
-
-    public StringCastEvaluator(final Evaluator<?> subjectEvaluator) {
-        this.subjectEvaluator = subjectEvaluator;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final QueryResult<?> result = subjectEvaluator.evaluate(attributes);
-        if (result.getValue() == null) {
-            return new StringQueryResult(null);
-        }
-
-        return new StringQueryResult(String.valueOf(result.getValue()));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subjectEvaluator;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AndEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AndEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AndEvaluator.java
deleted file mode 100644
index 9ed8f06..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AndEvaluator.java
+++ /dev/null
@@ -1,60 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class AndEvaluator extends BooleanEvaluator {
-
-    private final BooleanEvaluator subjectEvaluator;
-    private final BooleanEvaluator rhsEvaluator;
-
-    public AndEvaluator(final BooleanEvaluator subjectEvaluator, final BooleanEvaluator rhsEvaluator) {
-        this.subjectEvaluator = subjectEvaluator;
-        this.rhsEvaluator = rhsEvaluator;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final QueryResult<Boolean> subjectValue = subjectEvaluator.evaluate(attributes);
-        if (subjectValue == null) {
-            return new BooleanQueryResult(null);
-        }
-
-        if (Boolean.FALSE.equals(subjectValue.getValue())) {
-            return new BooleanQueryResult(false);
-        }
-
-        final QueryResult<Boolean> rhsValue = rhsEvaluator.evaluate(attributes);
-        if (rhsValue == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        return new BooleanQueryResult(rhsValue.getValue());
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subjectEvaluator;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AppendEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AppendEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AppendEvaluator.java
deleted file mode 100644
index bd5e81a..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AppendEvaluator.java
+++ /dev/null
@@ -1,50 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class AppendEvaluator extends StringEvaluator {
-
-    private final StringEvaluator subject;
-    private final StringEvaluator appendEvaluator;
-
-    public AppendEvaluator(final StringEvaluator subject, final StringEvaluator append) {
-        this.subject = subject;
-        this.appendEvaluator = append;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        final String appendValue = appendEvaluator.evaluate(attributes).getValue();
-
-        final String result = (subjectValue == null ? "" : subjectValue)
-                + (appendValue == null ? "" : appendValue);
-        return new StringQueryResult(result);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java
deleted file mode 100644
index 1b4b74e..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java
+++ /dev/null
@@ -1,45 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class AttributeEvaluator extends StringEvaluator {
-
-    private final StringEvaluator nameEvaluator;
-
-    public AttributeEvaluator(final StringEvaluator nameEvaluator) {
-        this.nameEvaluator = nameEvaluator;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String nameValue = nameEvaluator.evaluate(attributes).getValue();
-        final String attributeValue = attributes.get(nameValue);
-        return new StringQueryResult(attributeValue);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ContainsEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ContainsEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ContainsEvaluator.java
deleted file mode 100644
index 99802f6..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ContainsEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-
-public class ContainsEvaluator extends BooleanEvaluator {
-
-    private final StringEvaluator subject;
-    private final StringEvaluator search;
-
-    public ContainsEvaluator(final StringEvaluator subject, final StringEvaluator search) {
-        this.subject = subject;
-        this.search = search;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        final String searchString = search.evaluate(attributes).getValue();
-        return new BooleanQueryResult(searchString == null ? false : subjectValue.contains(searchString));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DateToNumberEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DateToNumberEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DateToNumberEvaluator.java
deleted file mode 100644
index 0594c4a..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DateToNumberEvaluator.java
+++ /dev/null
@@ -1,50 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Date;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class DateToNumberEvaluator extends NumberEvaluator {
-
-    private final DateEvaluator subjectEvaluator;
-
-    public DateToNumberEvaluator(final DateEvaluator subjectEvaluator) {
-        this.subjectEvaluator = subjectEvaluator;
-    }
-
-    @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final QueryResult<Date> subjectResult = subjectEvaluator.evaluate(attributes);
-        if (subjectResult.getValue() == null) {
-            return new NumberQueryResult(null);
-        }
-
-        return new NumberQueryResult(subjectResult.getValue().getTime());
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subjectEvaluator;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DivideEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DivideEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DivideEvaluator.java
deleted file mode 100644
index c7583cb..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DivideEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class DivideEvaluator extends NumberEvaluator {
-
-    private final NumberEvaluator subject;
-    private final NumberEvaluator divideValue;
-
-    public DivideEvaluator(final NumberEvaluator subject, final NumberEvaluator divideValue) {
-        this.subject = subject;
-        this.divideValue = divideValue;
-    }
-
-    @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final Long subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new NumberQueryResult(null);
-        }
-
-        final Long divide = divideValue.evaluate(attributes).getValue();
-        if (divide == null) {
-            return new NumberQueryResult(null);
-        }
-
-        final long result = subjectValue / divide;
-        return new NumberQueryResult(result);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EndsWithEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EndsWithEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EndsWithEvaluator.java
deleted file mode 100644
index c31c2b1..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EndsWithEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-
-public class EndsWithEvaluator extends BooleanEvaluator {
-
-    private final StringEvaluator subject;
-    private final StringEvaluator search;
-
-    public EndsWithEvaluator(final StringEvaluator subject, final StringEvaluator search) {
-        this.subject = subject;
-        this.search = search;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        final String searchString = search.evaluate(attributes).getValue();
-        return new BooleanQueryResult(searchString == null ? false : subjectValue.endsWith(searchString));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsEvaluator.java
deleted file mode 100644
index ec67b0f..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsEvaluator.java
+++ /dev/null
@@ -1,89 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Date;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-
-public class EqualsEvaluator extends BooleanEvaluator {
-
-    private final Evaluator<?> subject;
-    private final Evaluator<?> compareTo;
-
-    public EqualsEvaluator(final Evaluator<?> subject, final Evaluator<?> compareTo) {
-        this.subject = subject;
-        this.compareTo = compareTo;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Object a = subject.evaluate(attributes).getValue();
-        if (a == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        final Object b = compareTo.evaluate(attributes).getValue();
-        if (b == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        if (subject.getResultType() == compareTo.getResultType()) {
-            return new BooleanQueryResult(a.equals(b));
-        }
-
-        final String normalizedSubjectValue = normalizeValue(subject.getResultType(), a);
-        if (normalizedSubjectValue == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        final String normalizedCompareToValue = normalizeValue(compareTo.getResultType(), b);
-        if (normalizedCompareToValue == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        return new BooleanQueryResult(normalizedSubjectValue.equals(normalizedCompareToValue));
-    }
-
-    private String normalizeValue(final ResultType type, final Object value) {
-        if (value == null) {
-            return null;
-        }
-
-        switch (type) {
-            case STRING:
-                return (String) value;
-            case DATE:
-                return String.valueOf(((Date) value).getTime());
-            case BOOLEAN:
-            case NUMBER:
-            default:
-                return String.valueOf(value);
-        }
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsIgnoreCaseEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsIgnoreCaseEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsIgnoreCaseEvaluator.java
deleted file mode 100644
index 344395b..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsIgnoreCaseEvaluator.java
+++ /dev/null
@@ -1,60 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class EqualsIgnoreCaseEvaluator extends BooleanEvaluator {
-
-    private final Evaluator<?> subject;
-    private final Evaluator<?> compareTo;
-
-    public EqualsIgnoreCaseEvaluator(final Evaluator<?> subject, final Evaluator<?> compareTo) {
-        this.subject = subject;
-        this.compareTo = compareTo;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Object a = subject.evaluate(attributes).getValue();
-        if (a == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        final Object b = compareTo.evaluate(attributes).getValue();
-        if (b == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        if (a instanceof String && b instanceof String) {
-            return new BooleanQueryResult(((String) a).equalsIgnoreCase((String) b));
-        }
-
-        return new BooleanQueryResult(a.equals(b));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java
deleted file mode 100644
index b377c4a..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java
+++ /dev/null
@@ -1,72 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-import java.util.regex.Pattern;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator;
-
-public class FindEvaluator extends BooleanEvaluator {
-
-    private final StringEvaluator subject;
-    private final StringEvaluator search;
-
-    private final Pattern compiledPattern;
-
-    public FindEvaluator(final StringEvaluator subject, final StringEvaluator search) {
-        this.subject = subject;
-        this.search = search;
-
-        // if the search string is a literal, we don't need to evaluate it each time; we can just
-        // pre-compile it. Otherwise, it must be compiled every time.
-        if (search instanceof StringLiteralEvaluator) {
-            this.compiledPattern = Pattern.compile(search.evaluate(null).getValue());
-        } else {
-            this.compiledPattern = null;
-        }
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new BooleanQueryResult(false);
-        }
-        final Pattern pattern;
-        if (compiledPattern == null) {
-            pattern = Pattern.compile(search.evaluate(attributes).getValue());
-        } else {
-            pattern = compiledPattern;
-        }
-
-        final boolean found = pattern.matcher(subjectValue).find();
-
-        return new BooleanQueryResult(found);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FormatEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FormatEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FormatEvaluator.java
deleted file mode 100644
index 132b9e7..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FormatEvaluator.java
+++ /dev/null
@@ -1,61 +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.attribute.expression.language.evaluation.functions;
-
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Locale;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class FormatEvaluator extends StringEvaluator {
-
-    private final DateEvaluator subject;
-    private final StringEvaluator format;
-
-    public FormatEvaluator(final DateEvaluator subject, final StringEvaluator format) {
-        this.subject = subject;
-        this.format = format;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final Date subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new StringQueryResult(null);
-        }
-
-        final QueryResult<String> formatResult = format.evaluate(attributes);
-        final String format = formatResult.getValue();
-        if (format == null) {
-            return null;
-        }
-
-        return new StringQueryResult(new SimpleDateFormat(format, Locale.US).format(subjectValue));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanEvaluator.java
deleted file mode 100644
index 6c712bb..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanEvaluator.java
+++ /dev/null
@@ -1,60 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class GreaterThanEvaluator extends BooleanEvaluator {
-
-    private final NumberEvaluator subject;
-    private final NumberEvaluator comparison;
-
-    public GreaterThanEvaluator(final NumberEvaluator subject, final NumberEvaluator comparison) {
-        this.subject = subject;
-        this.comparison = comparison;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Long subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        final Long comparisonValue = comparison.evaluate(attributes).getValue();
-        if (comparisonValue == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        return new BooleanQueryResult(subjectValue > comparisonValue);
-    }
-
-    ;
-    
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanOrEqualEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanOrEqualEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanOrEqualEvaluator.java
deleted file mode 100644
index 98951f2..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanOrEqualEvaluator.java
+++ /dev/null
@@ -1,60 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class GreaterThanOrEqualEvaluator extends BooleanEvaluator {
-
-    private final NumberEvaluator subject;
-    private final NumberEvaluator comparison;
-
-    public GreaterThanOrEqualEvaluator(final NumberEvaluator subject, final NumberEvaluator comparison) {
-        this.subject = subject;
-        this.comparison = comparison;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Long subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        final Long comparisonValue = comparison.evaluate(attributes).getValue();
-        if (comparisonValue == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        return new BooleanQueryResult(subjectValue >= comparisonValue);
-    }
-
-    ;
-    
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/HostnameEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/HostnameEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/HostnameEvaluator.java
deleted file mode 100644
index c0b1134..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/HostnameEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class HostnameEvaluator extends StringEvaluator {
-
-    private final StringQueryResult hostname;
-
-    public HostnameEvaluator(final boolean preferFQDN) throws UnknownHostException {
-        String address;
-
-        if (preferFQDN) {
-            try {
-                address = InetAddress.getLocalHost().getCanonicalHostName();
-            } catch (final Exception e) {
-                address = InetAddress.getLocalHost().getHostName();
-            }
-        } else {
-            address = InetAddress.getLocalHost().getHostName();
-        }
-
-        hostname = new StringQueryResult(address);
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        return hostname;
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IPEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IPEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IPEvaluator.java
deleted file mode 100644
index 21f9fe8..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IPEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class IPEvaluator extends StringEvaluator {
-
-    private final StringQueryResult ipAddress;
-
-    public IPEvaluator() throws UnknownHostException {
-        ipAddress = new StringQueryResult(InetAddress.getLocalHost().getHostAddress());
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        return ipAddress;
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IndexOfEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IndexOfEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IndexOfEvaluator.java
deleted file mode 100644
index 80422cb..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IndexOfEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-
-public class IndexOfEvaluator extends NumberEvaluator {
-
-    private final StringEvaluator subject;
-    private final StringEvaluator indexEvaluator;
-
-    public IndexOfEvaluator(final StringEvaluator subject, final StringEvaluator indexEvaluator) {
-        this.subject = subject;
-        this.indexEvaluator = indexEvaluator;
-    }
-
-    @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new NumberQueryResult(-1L);
-        }
-        final String indexEvalValue = indexEvaluator.evaluate(attributes).getValue();
-
-        return new NumberQueryResult((long) subjectValue.indexOf(indexEvalValue));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsEmptyEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsEmptyEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsEmptyEvaluator.java
deleted file mode 100644
index c5e3c21..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsEmptyEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class IsEmptyEvaluator extends BooleanEvaluator {
-    private final Evaluator<?> subjectEvaluator;
-    
-    public IsEmptyEvaluator(final Evaluator<?> subjectEvaluator) {
-        this.subjectEvaluator = subjectEvaluator;
-    }
-    
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Object subjectValue = subjectEvaluator.evaluate(attributes).getValue();
-        return new BooleanQueryResult(subjectValue == null || subjectValue.toString().trim().isEmpty());
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subjectEvaluator;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsNullEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsNullEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsNullEvaluator.java
deleted file mode 100644
index a9d678c..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsNullEvaluator.java
+++ /dev/null
@@ -1,45 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class IsNullEvaluator extends BooleanEvaluator {
-
-    private final Evaluator<?> subject;
-
-    public IsNullEvaluator(final Evaluator<?> subject) {
-        this.subject = subject;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Object subjectValue = subject.evaluate(attributes).getValue();
-        return new BooleanQueryResult(subjectValue == null);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LastIndexOfEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LastIndexOfEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LastIndexOfEvaluator.java
deleted file mode 100644
index 2b67093..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LastIndexOfEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-
-public class LastIndexOfEvaluator extends NumberEvaluator {
-
-    private final StringEvaluator subject;
-    private final StringEvaluator indexEvaluator;
-
-    public LastIndexOfEvaluator(final StringEvaluator subject, final StringEvaluator indexEvaluator) {
-        this.subject = subject;
-        this.indexEvaluator = indexEvaluator;
-    }
-
-    @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new NumberQueryResult(-1L);
-        }
-        final String indexEvalValue = indexEvaluator.evaluate(attributes).getValue();
-
-        return new NumberQueryResult((long) subjectValue.lastIndexOf(indexEvalValue));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LengthEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LengthEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LengthEvaluator.java
deleted file mode 100644
index dec1f8f..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LengthEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-
-public class LengthEvaluator extends NumberEvaluator {
-
-    private final StringEvaluator subject;
-
-    public LengthEvaluator(final StringEvaluator subject) {
-        this.subject = subject;
-    }
-
-    @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        return new NumberQueryResult((long) ((subjectValue == null) ? 0 : subjectValue.length()));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanEvaluator.java
deleted file mode 100644
index 9a58910..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanEvaluator.java
+++ /dev/null
@@ -1,60 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class LessThanEvaluator extends BooleanEvaluator {
-
-    private final NumberEvaluator subject;
-    private final NumberEvaluator comparison;
-
-    public LessThanEvaluator(final NumberEvaluator subject, final NumberEvaluator comparison) {
-        this.subject = subject;
-        this.comparison = comparison;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Long subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        final Long comparisonValue = comparison.evaluate(attributes).getValue();
-        if (comparisonValue == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        return new BooleanQueryResult(subjectValue < comparisonValue);
-    }
-
-    ;
-    
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanOrEqualEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanOrEqualEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanOrEqualEvaluator.java
deleted file mode 100644
index 10f3f6a..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanOrEqualEvaluator.java
+++ /dev/null
@@ -1,60 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class LessThanOrEqualEvaluator extends BooleanEvaluator {
-
-    private final NumberEvaluator subject;
-    private final NumberEvaluator comparison;
-
-    public LessThanOrEqualEvaluator(final NumberEvaluator subject, final NumberEvaluator comparison) {
-        this.subject = subject;
-        this.comparison = comparison;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Long subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        final Long comparisonValue = comparison.evaluate(attributes).getValue();
-        if (comparisonValue == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        return new BooleanQueryResult(subjectValue <= comparisonValue);
-    }
-
-    ;
-    
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java
deleted file mode 100644
index 1bb86a5..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java
+++ /dev/null
@@ -1,71 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-import java.util.regex.Pattern;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator;
-
-public class MatchesEvaluator extends BooleanEvaluator {
-
-    private final StringEvaluator subject;
-    private final StringEvaluator search;
-
-    private final Pattern compiledPattern;
-
-    public MatchesEvaluator(final StringEvaluator subject, final StringEvaluator search) {
-        this.subject = subject;
-        this.search = search;
-
-        // if the search string is a literal, we don't need to evaluate it each time; we can just
-        // pre-compile it. Otherwise, it must be compiled every time.
-        if (search instanceof StringLiteralEvaluator) {
-            this.compiledPattern = Pattern.compile(search.evaluate(null).getValue());
-        } else {
-            this.compiledPattern = null;
-        }
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new BooleanQueryResult(false);
-        }
-        final Pattern pattern;
-        if (compiledPattern == null) {
-            pattern = Pattern.compile(search.evaluate(attributes).getValue());
-        } else {
-            pattern = compiledPattern;
-        }
-
-        final boolean matches = pattern.matcher(subjectValue).matches();
-        return new BooleanQueryResult(matches);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MinusEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MinusEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MinusEvaluator.java
deleted file mode 100644
index 0665979..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MinusEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class MinusEvaluator extends NumberEvaluator {
-
-    private final NumberEvaluator subject;
-    private final NumberEvaluator minusValue;
-
-    public MinusEvaluator(final NumberEvaluator subject, final NumberEvaluator minusValue) {
-        this.subject = subject;
-        this.minusValue = minusValue;
-    }
-
-    @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final Long subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new NumberQueryResult(null);
-        }
-
-        final Long minus = minusValue.evaluate(attributes).getValue();
-        if (minus == null) {
-            return new NumberQueryResult(null);
-        }
-
-        final long result = subjectValue - minus;
-        return new NumberQueryResult(result);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}


[09/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java
deleted file mode 100644
index 59d2308..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java
+++ /dev/null
@@ -1,1096 +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;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.List;
-import java.util.PriorityQueue;
-import java.util.Queue;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import org.apache.nifi.connectable.Connection;
-import org.apache.nifi.controller.repository.FlowFileRecord;
-import org.apache.nifi.controller.repository.claim.ContentClaim;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.FlowFileFilter;
-import org.apache.nifi.processor.FlowFileFilter.FlowFileFilterResult;
-import org.apache.nifi.processor.QueueSize;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.concurrency.TimedLock;
-import org.apache.nifi.util.timebuffer.LongEntityAccess;
-import org.apache.nifi.util.timebuffer.TimedBuffer;
-import org.apache.nifi.util.timebuffer.TimestampedLong;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A FlowFileQueue is used to queue FlowFile objects that are awaiting further
- * processing. Must be thread safe.
- *
- * @author none
- */
-public final class StandardFlowFileQueue implements FlowFileQueue {
-
-    public static final int MAX_EXPIRED_RECORDS_PER_ITERATION = 100000;
-    public static final int SWAP_RECORD_POLL_SIZE = 10000;
-
-    // When we have very high contention on a FlowFile Queue, the writeLock quickly becomes the bottleneck. In order to avoid this,
-    // we keep track of how often we are obtaining the write lock. If we exceed some threshold, we start performing a Pre-fetch so that
-    // we can then poll many times without having to obtain the lock.
-    // If lock obtained an average of more than PREFETCH_POLL_THRESHOLD times per second in order to poll from queue for last 5 seconds, do a pre-fetch.
-    public static final int PREFETCH_POLL_THRESHOLD = 1000;
-    public static final int PRIORITIZED_PREFETCH_SIZE = 10;
-    public static final int UNPRIORITIZED_PREFETCH_SIZE = 1000;
-    private volatile int prefetchSize = UNPRIORITIZED_PREFETCH_SIZE; // when we pre-fetch, how many should we pre-fetch?
-
-    private static final Logger logger = LoggerFactory.getLogger(StandardFlowFileQueue.class);
-
-    private PriorityQueue<FlowFileRecord> activeQueue = null;
-    private long activeQueueContentSize = 0L;
-    private ArrayList<FlowFileRecord> swapQueue = null;
-
-    private int swappedRecordCount = 0;
-    private long swappedContentSize = 0L;
-    private String maximumQueueDataSize;
-    private long maximumQueueByteCount;
-    private boolean swapMode = false;
-    private long maximumQueueObjectCount;
-
-    private final AtomicLong flowFileExpirationMillis;
-    private final Connection connection;
-    private final AtomicReference<String> flowFileExpirationPeriod;
-    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
-    private final List<FlowFilePrioritizer> priorities;
-    private final int swapThreshold;
-    private final TimedLock readLock;
-    private final TimedLock writeLock;
-    private final String identifier;
-
-    private final AtomicBoolean queueFullRef = new AtomicBoolean(false);
-    private final AtomicInteger activeQueueSizeRef = new AtomicInteger(0);
-    private final AtomicReference<QueueSize> unacknowledgedSizeRef = new AtomicReference<>(new QueueSize(0, 0L));
-
-    // SCHEDULER CANNOT BE NOTIFIED OF EVENTS WITH THE WRITE LOCK HELD! DOING SO WILL RESULT IN A DEADLOCK!
-    private final ProcessScheduler scheduler;
-
-    public StandardFlowFileQueue(final String identifier, final Connection connection, final ProcessScheduler scheduler, final int swapThreshold) {
-        activeQueue = new PriorityQueue<>(20, new Prioritizer(new ArrayList<FlowFilePrioritizer>()));
-        priorities = new ArrayList<>();
-        maximumQueueObjectCount = 0L;
-        maximumQueueDataSize = "0 MB";
-        maximumQueueByteCount = 0L;
-        flowFileExpirationMillis = new AtomicLong(0);
-        flowFileExpirationPeriod = new AtomicReference<>("0 mins");
-        swapQueue = new ArrayList<>();
-
-        this.identifier = identifier;
-        this.swapThreshold = swapThreshold;
-        this.scheduler = scheduler;
-        this.connection = connection;
-
-        readLock = new TimedLock(this.lock.readLock(), identifier + " Read Lock", 100);
-        writeLock = new TimedLock(this.lock.writeLock(), identifier + " Write Lock", 100);
-    }
-
-    @Override
-    public String getIdentifier() {
-        return identifier;
-    }
-
-    @Override
-    public List<FlowFilePrioritizer> getPriorities() {
-        return Collections.unmodifiableList(priorities);
-    }
-
-    @Override
-    public int getSwapThreshold() {
-        return swapThreshold;
-    }
-
-    @Override
-    public void setPriorities(final List<FlowFilePrioritizer> newPriorities) {
-        writeLock.lock();
-        try {
-            final PriorityQueue<FlowFileRecord> newQueue = new PriorityQueue<>(Math.max(20, activeQueue.size()), new Prioritizer(newPriorities));
-            newQueue.addAll(activeQueue);
-            activeQueue = newQueue;
-            priorities.clear();
-            priorities.addAll(newPriorities);
-
-            if (newPriorities.isEmpty()) {
-                prefetchSize = UNPRIORITIZED_PREFETCH_SIZE;
-            } else {
-                prefetchSize = PRIORITIZED_PREFETCH_SIZE;
-            }
-        } finally {
-            writeLock.unlock("setPriorities");
-        }
-    }
-
-    @Override
-    public void setBackPressureObjectThreshold(final long maxQueueSize) {
-        writeLock.lock();
-        try {
-            maximumQueueObjectCount = maxQueueSize;
-            this.queueFullRef.set(determineIfFull());
-        } finally {
-            writeLock.unlock("setBackPressureObjectThreshold");
-        }
-    }
-
-    @Override
-    public long getBackPressureObjectThreshold() {
-        readLock.lock();
-        try {
-            return maximumQueueObjectCount;
-        } finally {
-            readLock.unlock("getBackPressureObjectThreshold");
-        }
-    }
-
-    @Override
-    public void setBackPressureDataSizeThreshold(final String maxDataSize) {
-        writeLock.lock();
-        try {
-            maximumQueueByteCount = DataUnit.parseDataSize(maxDataSize, DataUnit.B).longValue();
-            maximumQueueDataSize = maxDataSize;
-            this.queueFullRef.set(determineIfFull());
-        } finally {
-            writeLock.unlock("setBackPressureDataSizeThreshold");
-        }
-    }
-
-    @Override
-    public String getBackPressureDataSizeThreshold() {
-        readLock.lock();
-        try {
-            return maximumQueueDataSize;
-        } finally {
-            readLock.unlock("getBackPressureDataSizeThreshold");
-        }
-    }
-
-    @Override
-    public QueueSize size() {
-        readLock.lock();
-        try {
-            return getQueueSize();
-        } finally {
-            readLock.unlock("getSize");
-        }
-    }
-
-    /**
-     * MUST be called with lock held
-     *
-     * @return
-     */
-    private QueueSize getQueueSize() {
-        final QueueSize unacknowledged = unacknowledgedSizeRef.get();
-        final PreFetch preFetch = preFetchRef.get();
-
-        final int preFetchCount;
-        final long preFetchSize;
-        if (preFetch == null) {
-            preFetchCount = 0;
-            preFetchSize = 0L;
-        } else {
-            final QueueSize preFetchQueueSize = preFetch.size();
-            preFetchCount = preFetchQueueSize.getObjectCount();
-            preFetchSize = preFetchQueueSize.getByteCount();
-        }
-
-        return new QueueSize(activeQueue.size() + swappedRecordCount + unacknowledged.getObjectCount() + preFetchCount,
-                activeQueueContentSize + swappedContentSize + unacknowledged.getByteCount() + preFetchSize);
-    }
-
-    @Override
-    public long contentSize() {
-        readLock.lock();
-        try {
-            final PreFetch prefetch = preFetchRef.get();
-            if (prefetch == null) {
-                return activeQueueContentSize + swappedContentSize + unacknowledgedSizeRef.get().getObjectCount();
-            } else {
-                return activeQueueContentSize + swappedContentSize + unacknowledgedSizeRef.get().getObjectCount() + prefetch.size().getByteCount();
-            }
-        } finally {
-            readLock.unlock("getContentSize");
-        }
-    }
-
-    @Override
-    public boolean isEmpty() {
-        readLock.lock();
-        try {
-            final PreFetch prefetch = preFetchRef.get();
-            if (prefetch == null) {
-                return activeQueue.isEmpty() && swappedRecordCount == 0 && unacknowledgedSizeRef.get().getObjectCount() == 0;
-            } else {
-                return activeQueue.isEmpty() && swappedRecordCount == 0 && unacknowledgedSizeRef.get().getObjectCount() == 0 && prefetch.size().getObjectCount() == 0;
-            }
-        } finally {
-            readLock.unlock("isEmpty");
-        }
-    }
-
-    @Override
-    public boolean isActiveQueueEmpty() {
-        final int activeQueueSize = activeQueueSizeRef.get();
-        if (activeQueueSize == 0) {
-            final PreFetch preFetch = preFetchRef.get();
-            if (preFetch == null) {
-                return true;
-            }
-
-            final QueueSize queueSize = preFetch.size();
-            return queueSize.getObjectCount() == 0;
-        } else {
-            return false;
-        }
-    }
-
-    @Override
-    public QueueSize getActiveQueueSize() {
-        readLock.lock();
-        try {
-            final PreFetch preFetch = preFetchRef.get();
-            if (preFetch == null) {
-                return new QueueSize(activeQueue.size(), activeQueueContentSize);
-            } else {
-                final QueueSize preFetchSize = preFetch.size();
-                return new QueueSize(activeQueue.size() + preFetchSize.getObjectCount(), activeQueueContentSize + preFetchSize.getByteCount());
-            }
-        } finally {
-            readLock.unlock("getActiveQueueSize");
-        }
-    }
-
-    @Override
-    public void acknowledge(final FlowFileRecord flowFile) {
-        if (queueFullRef.get()) {
-            writeLock.lock();
-            try {
-                updateUnacknowledgedSize(-1, -flowFile.getSize());
-                queueFullRef.set(determineIfFull());
-            } finally {
-                writeLock.unlock("acknowledge(FlowFileRecord)");
-            }
-        } else {
-            updateUnacknowledgedSize(-1, -flowFile.getSize());
-        }
-
-        if (connection.getSource().getSchedulingStrategy() == SchedulingStrategy.EVENT_DRIVEN) {
-            // queue was full but no longer is. Notify that the source may now be available to run,
-            // because of back pressure caused by this queue.
-            scheduler.registerEvent(connection.getSource());
-        }
-    }
-
-    @Override
-    public void acknowledge(final Collection<FlowFileRecord> flowFiles) {
-        long totalSize = 0L;
-        for (final FlowFileRecord flowFile : flowFiles) {
-            totalSize += flowFile.getSize();
-        }
-
-        if (queueFullRef.get()) {
-            writeLock.lock();
-            try {
-                updateUnacknowledgedSize(-flowFiles.size(), -totalSize);
-                queueFullRef.set(determineIfFull());
-            } finally {
-                writeLock.unlock("acknowledge(FlowFileRecord)");
-            }
-        } else {
-            updateUnacknowledgedSize(-flowFiles.size(), -totalSize);
-        }
-
-        if (connection.getSource().getSchedulingStrategy() == SchedulingStrategy.EVENT_DRIVEN) {
-            // it's possible that queue was full but no longer is. Notify that the source may now be available to run,
-            // because of back pressure caused by this queue.
-            scheduler.registerEvent(connection.getSource());
-        }
-    }
-
-    @Override
-    public boolean isFull() {
-        return queueFullRef.get();
-    }
-
-    /**
-     * MUST be called with either the read or write lock held
-     *
-     * @return
-     */
-    private boolean determineIfFull() {
-        final long maxSize = maximumQueueObjectCount;
-        final long maxBytes = maximumQueueByteCount;
-        if (maxSize <= 0 && maxBytes <= 0) {
-            return false;
-        }
-
-        final QueueSize queueSize = getQueueSize();
-        if (maxSize > 0 && queueSize.getObjectCount() >= maxSize) {
-            return true;
-        }
-
-        if (maxBytes > 0 && (queueSize.getByteCount() >= maxBytes)) {
-            return true;
-        }
-
-        return false;
-    }
-
-    @Override
-    public void put(final FlowFileRecord file) {
-        writeLock.lock();
-        try {
-            if (swapMode || activeQueue.size() >= swapThreshold) {
-                swapQueue.add(file);
-                swappedContentSize += file.getSize();
-                swappedRecordCount++;
-                swapMode = true;
-            } else {
-                activeQueueContentSize += file.getSize();
-                activeQueue.add(file);
-            }
-
-            queueFullRef.set(determineIfFull());
-        } finally {
-            activeQueueSizeRef.set(activeQueue.size());
-            writeLock.unlock("put(FlowFileRecord)");
-        }
-
-        if (connection.getDestination().getSchedulingStrategy() == SchedulingStrategy.EVENT_DRIVEN) {
-            scheduler.registerEvent(connection.getDestination());
-        }
-    }
-
-    @Override
-    public void putAll(final Collection<FlowFileRecord> files) {
-        final int numFiles = files.size();
-        long bytes = 0L;
-        for (final FlowFile flowFile : files) {
-            bytes += flowFile.getSize();
-        }
-
-        writeLock.lock();
-        try {
-            if (swapMode || activeQueue.size() >= swapThreshold - numFiles) {
-                swapQueue.addAll(files);
-                swappedContentSize += bytes;
-                swappedRecordCount += numFiles;
-                swapMode = true;
-            } else {
-                activeQueueContentSize += bytes;
-                activeQueue.addAll(files);
-            }
-
-            queueFullRef.set(determineIfFull());
-        } finally {
-            activeQueueSizeRef.set(activeQueue.size());
-            writeLock.unlock("putAll");
-        }
-
-        if (connection.getDestination().getSchedulingStrategy() == SchedulingStrategy.EVENT_DRIVEN) {
-            scheduler.registerEvent(connection.getDestination());
-        }
-    }
-
-    @Override
-    public List<FlowFileRecord> pollSwappableRecords() {
-        writeLock.lock();
-        try {
-            if (swapQueue.size() < SWAP_RECORD_POLL_SIZE) {
-                return null;
-            }
-
-            final List<FlowFileRecord> swapRecords = new ArrayList<>(Math.min(SWAP_RECORD_POLL_SIZE, swapQueue.size()));
-            final Iterator<FlowFileRecord> itr = swapQueue.iterator();
-            while (itr.hasNext() && swapRecords.size() < SWAP_RECORD_POLL_SIZE) {
-                FlowFileRecord record = itr.next();
-                swapRecords.add(record);
-                itr.remove();
-            }
-
-            swapQueue.trimToSize();
-            return swapRecords;
-        } finally {
-            writeLock.unlock("pollSwappableRecords");
-        }
-    }
-
-    @Override
-    public void putSwappedRecords(final Collection<FlowFileRecord> records) {
-        writeLock.lock();
-        try {
-            try {
-                for (final FlowFileRecord record : records) {
-                    swappedContentSize -= record.getSize();
-                    swappedRecordCount--;
-                    activeQueueContentSize += record.getSize();
-                    activeQueue.add(record);
-                }
-
-                if (swappedRecordCount > swapQueue.size()) {
-                    // we have more swap files to be swapped in.
-                    return;
-                }
-
-                // If a call to #pollSwappableRecords will not produce any, go ahead and roll those FlowFiles back into the mix
-                if (swapQueue.size() < SWAP_RECORD_POLL_SIZE) {
-                    for (final FlowFileRecord record : swapQueue) {
-                        activeQueue.add(record);
-                        activeQueueContentSize += record.getSize();
-                    }
-                    swapQueue.clear();
-                    swappedContentSize = 0L;
-                    swappedRecordCount = 0;
-                    swapMode = false;
-                }
-            } finally {
-                activeQueueSizeRef.set(activeQueue.size());
-            }
-        } finally {
-            writeLock.unlock("putSwappedRecords");
-            scheduler.registerEvent(connection.getDestination());
-        }
-    }
-
-    @Override
-    public void incrementSwapCount(final int numRecords, final long contentSize) {
-        writeLock.lock();
-        try {
-            swappedContentSize += contentSize;
-            swappedRecordCount += numRecords;
-        } finally {
-            writeLock.unlock("incrementSwapCount");
-        }
-    }
-
-    @Override
-    public int unswappedSize() {
-        readLock.lock();
-        try {
-            return activeQueue.size() + unacknowledgedSizeRef.get().getObjectCount();
-        } finally {
-            readLock.unlock("unswappedSize");
-        }
-    }
-
-    @Override
-    public int getSwapRecordCount() {
-        readLock.lock();
-        try {
-            return swappedRecordCount;
-        } finally {
-            readLock.unlock("getSwapRecordCount");
-        }
-    }
-
-    @Override
-    public int getSwapQueueSize() {
-        readLock.lock();
-        try {
-            if (logger.isDebugEnabled()) {
-                final long byteToMbDivisor = 1024L * 1024L;
-                final QueueSize unacknowledged = unacknowledgedSizeRef.get();
-
-                logger.debug("Total Queue Size: ActiveQueue={}/{} MB, Swap Queue={}/{} MB, Unacknowledged={}/{} MB",
-                        activeQueue.size(), activeQueueContentSize / byteToMbDivisor,
-                        swappedRecordCount, swappedContentSize / byteToMbDivisor,
-                        unacknowledged.getObjectCount(), unacknowledged.getByteCount() / byteToMbDivisor);
-            }
-
-            return swapQueue.size();
-        } finally {
-            readLock.unlock("getSwapQueueSize");
-        }
-    }
-
-    private boolean isLaterThan(final Long maxAge) {
-        if (maxAge == null) {
-            return false;
-        }
-        return maxAge < System.currentTimeMillis();
-    }
-
-    private Long getExpirationDate(final FlowFile flowFile, final long expirationMillis) {
-        if (flowFile == null) {
-            return null;
-        }
-        if (expirationMillis <= 0) {
-            return null;
-        } else {
-            final long entryDate = flowFile.getEntryDate();
-            final long expirationDate = entryDate + expirationMillis;
-            return expirationDate;
-        }
-    }
-
-    @Override
-    public FlowFileRecord poll(final Set<FlowFileRecord> expiredRecords) {
-        FlowFileRecord flowFile = null;
-
-        // First check if we have any records Pre-Fetched.
-        final long expirationMillis = flowFileExpirationMillis.get();
-        final PreFetch preFetch = preFetchRef.get();
-        if (preFetch != null) {
-            if (preFetch.isExpired()) {
-                requeueExpiredPrefetch(preFetch);
-            } else {
-                while (true) {
-                    final FlowFileRecord next = preFetch.nextRecord();
-                    if (next == null) {
-                        break;
-                    }
-
-                    if (isLaterThan(getExpirationDate(next, expirationMillis))) {
-                        expiredRecords.add(next);
-                        continue;
-                    }
-
-                    updateUnacknowledgedSize(1, next.getSize());
-                    return next;
-                }
-
-                preFetchRef.compareAndSet(preFetch, null);
-            }
-        }
-
-        writeLock.lock();
-        try {
-            flowFile = doPoll(expiredRecords, expirationMillis);
-            return flowFile;
-        } finally {
-            activeQueueSizeRef.set(activeQueue.size());
-            writeLock.unlock("poll(Set)");
-
-            if (flowFile != null) {
-                updateUnacknowledgedSize(1, flowFile.getSize());
-            }
-        }
-    }
-
-    private FlowFileRecord doPoll(final Set<FlowFileRecord> expiredRecords, final long expirationMillis) {
-        FlowFileRecord flowFile;
-        boolean isExpired;
-
-        migrateSwapToActive();
-        boolean queueFullAtStart = queueFullRef.get();
-
-        do {
-            flowFile = this.activeQueue.poll();
-
-            isExpired = isLaterThan(getExpirationDate(flowFile, expirationMillis));
-            if (isExpired) {
-                expiredRecords.add(flowFile);
-                if (expiredRecords.size() >= MAX_EXPIRED_RECORDS_PER_ITERATION) {
-                    activeQueueContentSize -= flowFile.getSize();
-                    break;
-                }
-            } else if (flowFile != null && flowFile.isPenalized()) {
-                this.activeQueue.add(flowFile);
-                flowFile = null;
-                break;
-            }
-
-            if (flowFile != null) {
-                activeQueueContentSize -= flowFile.getSize();
-            }
-        } while (isExpired);
-
-        // if at least 1 FlowFile was expired & the queue was full before we started, then
-        // we need to determine whether or not the queue is full again. If no FlowFile was expired,
-        // then the queue will still be full until the appropriate #acknowledge method is called.
-        if (queueFullAtStart && !expiredRecords.isEmpty()) {
-            queueFullRef.set(determineIfFull());
-        }
-
-        if (incrementPollCount()) {
-            prefetch();
-        }
-        return isExpired ? null : flowFile;
-    }
-
-    @Override
-    public List<FlowFileRecord> poll(int maxResults, final Set<FlowFileRecord> expiredRecords) {
-        final List<FlowFileRecord> records = new ArrayList<>(Math.min(1024, maxResults));
-
-        // First check if we have any records Pre-Fetched.
-        final long expirationMillis = flowFileExpirationMillis.get();
-        final PreFetch preFetch = preFetchRef.get();
-        if (preFetch != null) {
-            if (preFetch.isExpired()) {
-                requeueExpiredPrefetch(preFetch);
-            } else {
-                long totalSize = 0L;
-                for (int i = 0; i < maxResults; i++) {
-                    final FlowFileRecord next = preFetch.nextRecord();
-                    if (next == null) {
-                        break;
-                    }
-
-                    if (isLaterThan(getExpirationDate(next, expirationMillis))) {
-                        expiredRecords.add(next);
-                        continue;
-                    }
-
-                    records.add(next);
-                    totalSize += next.getSize();
-                }
-
-                // If anything was prefetched, use what we have.
-                if (!records.isEmpty()) {
-                    updateUnacknowledgedSize(records.size(), totalSize);
-                    return records;
-                }
-
-                preFetchRef.compareAndSet(preFetch, null);
-            }
-        }
-
-        writeLock.lock();
-        try {
-            doPoll(records, maxResults, expiredRecords);
-        } finally {
-            activeQueueSizeRef.set(activeQueue.size());
-            writeLock.unlock("poll(int, Set)");
-        }
-        return records;
-    }
-
-    private void doPoll(final List<FlowFileRecord> records, int maxResults, final Set<FlowFileRecord> expiredRecords) {
-        migrateSwapToActive();
-
-        final boolean queueFullAtStart = queueFullRef.get();
-
-        final long bytesDrained = drainQueue(activeQueue, records, maxResults, expiredRecords);
-
-        long expiredBytes = 0L;
-        for (final FlowFileRecord record : expiredRecords) {
-            expiredBytes += record.getSize();
-        }
-
-        activeQueueContentSize -= bytesDrained;
-        updateUnacknowledgedSize(records.size(), bytesDrained - expiredBytes);
-
-        // if at least 1 FlowFile was expired & the queue was full before we started, then
-        // we need to determine whether or not the queue is full again. If no FlowFile was expired,
-        // then the queue will still be full until the appropriate #acknowledge method is called.
-        if (queueFullAtStart && !expiredRecords.isEmpty()) {
-            queueFullRef.set(determineIfFull());
-        }
-
-        if (incrementPollCount()) {
-            prefetch();
-        }
-    }
-
-    /**
-     * If there are FlowFiles waiting on the swap queue, move them to the active
-     * queue until we meet our threshold. This prevents us from having to swap
-     * them to disk & then back out.
-     *
-     * This method MUST be called with the writeLock held.
-     */
-    private void migrateSwapToActive() {
-        // Migrate as many FlowFiles as we can from the Swap Queue to the Active Queue, so that we don't
-        // have to swap them out & then swap them back in.
-        // If we don't do this, we could get into a situation where we have potentially thousands of FlowFiles
-        // sitting on the Swap Queue but not getting processed because there aren't enough to be swapped out.
-        // In particular, this can happen if the queue is typically filled with surges.
-        // For example, if the queue has 25,000 FlowFiles come in, it may process 20,000 of them and leave
-        // 5,000 sitting on the Swap Queue. If it then takes an hour for an additional 5,000 FlowFiles to come in,
-        // those FlowFiles sitting on the Swap Queue will sit there for an hour, waiting to be swapped out and
-        // swapped back in again.
-        // Calling this method when records are polled prevents this condition by migrating FlowFiles from the
-        // Swap Queue to the Active Queue. However, we don't do this if there are FlowFiles already swapped out
-        // to disk, because we want them to be swapped back in in the same order that they were swapped out.
-
-        // this is the most common condition (nothing is swapped out), so do the check first and avoid the expense
-        // of other checks for 99.999% of the cases.
-        if (swappedRecordCount == 0 && swapQueue.isEmpty()) {
-            return;
-        }
-
-        if (swappedRecordCount > swapQueue.size()) {
-            // we already have FlowFiles swapped out, so we won't migrate the queue; we will wait for
-            // an external process to swap FlowFiles back in.
-            return;
-        }
-
-        final Iterator<FlowFileRecord> swapItr = swapQueue.iterator();
-        while (activeQueue.size() < swapThreshold && swapItr.hasNext()) {
-            final FlowFileRecord toMigrate = swapItr.next();
-            activeQueue.add(toMigrate);
-            activeQueueContentSize += toMigrate.getSize();
-            swappedContentSize -= toMigrate.getSize();
-            swappedRecordCount--;
-
-            swapItr.remove();
-        }
-
-        if (swappedRecordCount == 0) {
-            swapMode = false;
-        }
-    }
-
-    @Override
-    public long drainQueue(final Queue<FlowFileRecord> sourceQueue, final List<FlowFileRecord> destination, int maxResults, final Set<FlowFileRecord> expiredRecords) {
-        long drainedSize = 0L;
-        FlowFileRecord pulled = null;
-
-        final long expirationMillis = this.flowFileExpirationMillis.get();
-        while (destination.size() < maxResults && (pulled = sourceQueue.poll()) != null) {
-            if (isLaterThan(getExpirationDate(pulled, expirationMillis))) {
-                expiredRecords.add(pulled);
-                if (expiredRecords.size() >= MAX_EXPIRED_RECORDS_PER_ITERATION) {
-                    break;
-                }
-            } else {
-                if (pulled.isPenalized()) {
-                    sourceQueue.add(pulled);
-                    break;
-                }
-                destination.add(pulled);
-            }
-            drainedSize += pulled.getSize();
-        }
-        return drainedSize;
-    }
-
-    @Override
-    public List<FlowFileRecord> poll(final FlowFileFilter filter, final Set<FlowFileRecord> expiredRecords) {
-        writeLock.lock();
-        try {
-            migrateSwapToActive();
-            if (activeQueue.isEmpty()) {
-                return Collections.emptyList();
-            }
-
-            final long expirationMillis = this.flowFileExpirationMillis.get();
-            final boolean queueFullAtStart = queueFullRef.get();
-
-            final List<FlowFileRecord> selectedFlowFiles = new ArrayList<>();
-            final List<FlowFileRecord> unselected = new ArrayList<>();
-
-            while (true) {
-                FlowFileRecord flowFile = this.activeQueue.poll();
-                if (flowFile == null) {
-                    break;
-                }
-
-                final boolean isExpired = isLaterThan(getExpirationDate(flowFile, expirationMillis));
-                if (isExpired) {
-                    expiredRecords.add(flowFile);
-                    activeQueueContentSize -= flowFile.getSize();
-
-                    if (expiredRecords.size() >= MAX_EXPIRED_RECORDS_PER_ITERATION) {
-                        break;
-                    } else {
-                        continue;
-                    }
-                } else if (flowFile.isPenalized()) {
-                    this.activeQueue.add(flowFile);
-                    flowFile = null;
-                    break; // just stop searching because the rest are all penalized.
-                }
-
-                final FlowFileFilterResult result = filter.filter(flowFile);
-                if (result.isAccept()) {
-                    activeQueueContentSize -= flowFile.getSize();
-
-                    updateUnacknowledgedSize(1, flowFile.getSize());
-                    selectedFlowFiles.add(flowFile);
-                } else {
-                    unselected.add(flowFile);
-                }
-
-                if (!result.isContinue()) {
-                    break;
-                }
-            }
-
-            this.activeQueue.addAll(unselected);
-
-            // if at least 1 FlowFile was expired & the queue was full before we started, then
-            // we need to determine whether or not the queue is full again. If no FlowFile was expired,
-            // then the queue will still be full until the appropriate #acknowledge method is called.
-            if (queueFullAtStart && !expiredRecords.isEmpty()) {
-                queueFullRef.set(determineIfFull());
-            }
-
-            return selectedFlowFiles;
-        } finally {
-            activeQueueSizeRef.set(activeQueue.size());
-            writeLock.unlock("poll(Filter, Set)");
-        }
-    }
-
-    private static final class Prioritizer implements Comparator<FlowFileRecord>, Serializable {
-
-        private static final long serialVersionUID = 1L;
-        private final transient List<FlowFilePrioritizer> prioritizers = new ArrayList<>();
-
-        private Prioritizer(final List<FlowFilePrioritizer> priorities) {
-            if (null != priorities) {
-                prioritizers.addAll(priorities);
-            }
-        }
-
-        @Override
-        public int compare(final FlowFileRecord f1, final FlowFileRecord f2) {
-            int returnVal = 0;
-            final boolean f1Penalized = f1.isPenalized();
-            final boolean f2Penalized = f2.isPenalized();
-
-            if (f1Penalized && !f2Penalized) {
-                return 1;
-            } else if (!f1Penalized && f2Penalized) {
-                return -1;
-            }
-
-            if (f1Penalized && f2Penalized) {
-                if (f1.getPenaltyExpirationMillis() < f2.getPenaltyExpirationMillis()) {
-                    return -1;
-                } else if (f1.getPenaltyExpirationMillis() > f2.getPenaltyExpirationMillis()) {
-                    return 1;
-                }
-            }
-
-            if (!prioritizers.isEmpty()) {
-                for (final FlowFilePrioritizer prioritizer : prioritizers) {
-                    returnVal = prioritizer.compare(f1, f2);
-                    if (returnVal != 0) {
-                        return returnVal;
-                    }
-                }
-            }
-
-            final ContentClaim claim1 = f1.getContentClaim();
-            final ContentClaim claim2 = f2.getContentClaim();
-
-            // put the one without a claim first
-            if (claim1 == null && claim2 != null) {
-                return -1;
-            } else if (claim1 != null && claim2 == null) {
-                return 1;
-            } else if (claim1 != null && claim2 != null) {
-                final int claimComparison = claim1.compareTo(claim2);
-                if (claimComparison != 0) {
-                    return claimComparison;
-                }
-
-                final int claimOffsetComparison = Long.compare(f1.getContentClaimOffset(), f2.getContentClaimOffset());
-                if (claimOffsetComparison != 0) {
-                    return claimOffsetComparison;
-                }
-            }
-
-            return Long.compare(f1.getId(), f2.getId());
-        }
-    }
-
-    @Override
-    public String getFlowFileExpiration() {
-        return flowFileExpirationPeriod.get();
-    }
-
-    @Override
-    public int getFlowFileExpiration(final TimeUnit timeUnit) {
-        return (int) timeUnit.convert(flowFileExpirationMillis.get(), TimeUnit.MILLISECONDS);
-    }
-
-    @Override
-    public void setFlowFileExpiration(final String flowExpirationPeriod) {
-        final long millis = FormatUtils.getTimeDuration(flowExpirationPeriod, TimeUnit.MILLISECONDS);
-        if (millis < 0) {
-            throw new IllegalArgumentException("FlowFile Expiration Period must be positive");
-        }
-        this.flowFileExpirationPeriod.set(flowExpirationPeriod);
-        this.flowFileExpirationMillis.set(millis);
-    }
-
-    @Override
-    public String toString() {
-        return "FlowFileQueue[id=" + identifier + "]";
-    }
-
-    /**
-     * Lock the queue so that other threads are unable to interact with the
-     * queue
-     */
-    public void lock() {
-        writeLock.lock();
-    }
-
-    /**
-     * Unlock the queue
-     */
-    public void unlock() {
-        writeLock.unlock("external unlock");
-    }
-
-    private void updateUnacknowledgedSize(final int addToCount, final long addToSize) {
-        boolean updated = false;
-
-        do {
-            QueueSize queueSize = unacknowledgedSizeRef.get();
-            final QueueSize newSize = new QueueSize(queueSize.getObjectCount() + addToCount, queueSize.getByteCount() + addToSize);
-            updated = unacknowledgedSizeRef.compareAndSet(queueSize, newSize);
-        } while (!updated);
-    }
-
-    private void requeueExpiredPrefetch(final PreFetch prefetch) {
-        if (prefetch == null) {
-            return;
-        }
-
-        writeLock.lock();
-        try {
-            final long contentSizeRequeued = prefetch.requeue(activeQueue);
-            this.activeQueueContentSize += contentSizeRequeued;
-            this.preFetchRef.compareAndSet(prefetch, null);
-        } finally {
-            writeLock.unlock("requeueExpiredPrefetch");
-        }
-    }
-
-    /**
-     * MUST be called with write lock held.
-     */
-    private final AtomicReference<PreFetch> preFetchRef = new AtomicReference<>();
-
-    private void prefetch() {
-        if (activeQueue.isEmpty()) {
-            return;
-        }
-
-        final int numToFetch = Math.min(prefetchSize, activeQueue.size());
-
-        final PreFetch curPreFetch = preFetchRef.get();
-        if (curPreFetch != null && curPreFetch.size().getObjectCount() > 0) {
-            return;
-        }
-
-        final List<FlowFileRecord> buffer = new ArrayList<>(numToFetch);
-        long contentSize = 0L;
-        for (int i = 0; i < numToFetch; i++) {
-            final FlowFileRecord record = activeQueue.poll();
-            if (record == null || record.isPenalized()) {
-                // not enough unpenalized records to pull. Put all records back and return
-                activeQueue.addAll(buffer);
-                if ( record != null ) {
-                    activeQueue.add(record);
-                }
-                return;
-            } else {
-                buffer.add(record);
-                contentSize += record.getSize();
-            }
-        }
-
-        activeQueueContentSize -= contentSize;
-        preFetchRef.set(new PreFetch(buffer));
-    }
-
-    private final TimedBuffer<TimestampedLong> pollCounts = new TimedBuffer<>(TimeUnit.SECONDS, 5, new LongEntityAccess());
-
-    private boolean incrementPollCount() {
-        pollCounts.add(new TimestampedLong(1L));
-        final long totalCount = pollCounts.getAggregateValue(System.currentTimeMillis() - 5000L).getValue();
-        return totalCount > PREFETCH_POLL_THRESHOLD * 5;
-    }
-
-    private static class PreFetch {
-
-        private final List<FlowFileRecord> records;
-        private final AtomicInteger pointer = new AtomicInteger(0);
-        private final long expirationTime = System.nanoTime() + TimeUnit.SECONDS.toNanos(1L);
-        private final AtomicLong contentSize = new AtomicLong(0L);
-
-        public PreFetch(final List<FlowFileRecord> records) {
-            this.records = records;
-
-            long totalSize = 0L;
-            for (final FlowFileRecord record : records) {
-                totalSize += record.getSize();
-            }
-            contentSize.set(totalSize);
-        }
-
-        public FlowFileRecord nextRecord() {
-            final int nextValue = pointer.getAndIncrement();
-            if (nextValue >= records.size()) {
-                return null;
-            }
-
-            final FlowFileRecord flowFile = records.get(nextValue);
-            contentSize.addAndGet(-flowFile.getSize());
-            return flowFile;
-        }
-
-        public QueueSize size() {
-            final int pointerIndex = pointer.get();
-            final int count = records.size() - pointerIndex;
-            if (count < 0) {
-                return new QueueSize(0, 0L);
-            }
-
-            final long bytes = contentSize.get();
-            return new QueueSize(count, bytes);
-        }
-
-        public boolean isExpired() {
-            return System.nanoTime() > expirationTime;
-        }
-
-        private long requeue(final Queue<FlowFileRecord> queue) {
-            // get the current pointer and prevent any other thread from accessing the rest of the elements
-            final int curPointer = pointer.getAndAdd(records.size());
-            if (curPointer < records.size() - 1) {
-                final List<FlowFileRecord> subList = records.subList(curPointer, records.size());
-                long contentSize = 0L;
-                for (final FlowFileRecord record : subList) {
-                    contentSize += record.getSize();
-                }
-
-                queue.addAll(subList);
-
-                return contentSize;
-            }
-            return 0L;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
deleted file mode 100644
index e516f20..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
+++ /dev/null
@@ -1,541 +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;
-
-import static java.util.Objects.requireNonNull;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.connectable.Connectable;
-import org.apache.nifi.connectable.ConnectableType;
-import org.apache.nifi.connectable.Connection;
-import org.apache.nifi.connectable.Funnel;
-import org.apache.nifi.connectable.Position;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.groups.ProcessGroup;
-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.scheduling.SchedulingStrategy;
-import org.apache.nifi.util.FormatUtils;
-
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-public class StandardFunnel implements Funnel {
-
-    public static final long MINIMUM_PENALIZATION_MILLIS = 0L;
-    public static final TimeUnit DEFAULT_TIME_UNIT = TimeUnit.MILLISECONDS;
-    public static final long MINIMUM_YIELD_MILLIS = 0L;
-    public static final long DEFAULT_YIELD_PERIOD = 1000L;
-    public static final TimeUnit DEFAULT_YIELD_TIME_UNIT = TimeUnit.MILLISECONDS;
-
-    private final String identifier;
-    private final Set<Connection> outgoingConnections;
-    private final List<Connection> incomingConnections;
-    private final List<Relationship> relationships;
-
-    private final AtomicReference<ProcessGroup> processGroupRef;
-    private final AtomicReference<Position> position;
-    private final AtomicReference<String> penalizationPeriod;
-    private final AtomicReference<String> yieldPeriod;
-    private final AtomicReference<String> schedulingPeriod;
-    private final AtomicReference<String> name;
-    private final AtomicLong schedulingNanos;
-    private final AtomicBoolean lossTolerant;
-    private final AtomicReference<ScheduledState> scheduledState;
-    private final AtomicLong yieldExpiration;
-
-    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
-    private final Lock readLock = rwLock.readLock();
-    private final Lock writeLock = rwLock.writeLock();
-
-    public StandardFunnel(final String identifier, final ProcessGroup processGroup, final ProcessScheduler scheduler) {
-        this.identifier = identifier;
-        this.processGroupRef = new AtomicReference<>(processGroup);
-
-        outgoingConnections = new HashSet<>();
-        incomingConnections = new ArrayList<>();
-
-        final List<Relationship> relationships = new ArrayList<>();
-        relationships.add(Relationship.ANONYMOUS);
-        this.relationships = Collections.unmodifiableList(relationships);
-
-        lossTolerant = new AtomicBoolean(false);
-        position = new AtomicReference<>(new Position(0D, 0D));
-        scheduledState = new AtomicReference<>(ScheduledState.STOPPED);
-        penalizationPeriod = new AtomicReference<>("30 sec");
-        yieldPeriod = new AtomicReference<>("1 sec");
-        yieldExpiration = new AtomicLong(0L);
-        schedulingPeriod = new AtomicReference<>("0 millis");
-        schedulingNanos = new AtomicLong(30000);
-        name = new AtomicReference<>("Funnel");
-    }
-
-    @Override
-    public String getIdentifier() {
-        return identifier;
-    }
-
-    @Override
-    public Collection<Relationship> getRelationships() {
-        return relationships;
-    }
-
-    @Override
-    public Relationship getRelationship(final String relationshipName) {
-        return (Relationship.ANONYMOUS.getName().equals(relationshipName)) ? Relationship.ANONYMOUS : null;
-    }
-
-    @Override
-    public void addConnection(final Connection connection) throws IllegalArgumentException {
-        writeLock.lock();
-        try {
-            if (!requireNonNull(connection).getSource().equals(this) && !connection.getDestination().equals(this)) {
-                throw new IllegalArgumentException("Cannot add a connection to a Funnel for which the Funnel is neither the Source nor the Destination");
-            }
-            if (connection.getSource().equals(this) && connection.getDestination().equals(this)) {
-                throw new IllegalArgumentException("Cannot add a connection from a Funnel back to itself");
-            }
-
-            if (connection.getDestination().equals(this)) {
-                // don't add the connection twice. This may occur if we have a self-loop because we will be told
-                // to add the connection once because we are the source and again because we are the destination.
-                if (!incomingConnections.contains(connection)) {
-                    incomingConnections.add(connection);
-                }
-            }
-
-            if (connection.getSource().equals(this)) {
-                // don't add the connection twice. This may occur if we have a self-loop because we will be told
-                // to add the connection once because we are the source and again because we are the destination.
-                if (!outgoingConnections.contains(connection)) {
-                    for (final Relationship relationship : connection.getRelationships()) {
-                        if (!relationship.equals(Relationship.ANONYMOUS)) {
-                            throw new IllegalArgumentException("No relationship with name " + relationship + " exists for Funnels");
-                        }
-                    }
-
-                    outgoingConnections.add(connection);
-                }
-            }
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public boolean hasIncomingConnection() {
-        readLock.lock();
-        try {
-            return !incomingConnections.isEmpty();
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public void updateConnection(final Connection connection) throws IllegalStateException {
-        if (requireNonNull(connection).getSource().equals(this)) {
-            writeLock.lock();
-            try {
-                if (!outgoingConnections.remove(connection)) {
-                    throw new IllegalStateException("No Connection with ID " + connection.getIdentifier() + " is currently registered with this Port");
-                }
-                outgoingConnections.add(connection);
-            } finally {
-                writeLock.unlock();
-            }
-        }
-
-        if (connection.getDestination().equals(this)) {
-            writeLock.lock();
-            try {
-                if (!incomingConnections.remove(connection)) {
-                    throw new IllegalStateException("No Connection with ID " + connection.getIdentifier() + " is currently registered with this Port");
-                }
-                incomingConnections.add(connection);
-            } finally {
-                writeLock.unlock();
-            }
-        }
-    }
-
-    @Override
-    public void removeConnection(final Connection connection) throws IllegalArgumentException, IllegalStateException {
-        writeLock.lock();
-        try {
-            if (!requireNonNull(connection).getSource().equals(this)) {
-                final boolean existed = incomingConnections.remove(connection);
-                if (!existed) {
-                    throw new IllegalStateException("The given connection is not currently registered for this ProcessorNode");
-                }
-                return;
-            }
-
-            final boolean removed = outgoingConnections.remove(connection);
-            if (!removed) {
-                throw new IllegalStateException(connection + " is not registered with " + this);
-            }
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public Set<Connection> getConnections() {
-        readLock.lock();
-        try {
-            return Collections.unmodifiableSet(outgoingConnections);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public Set<Connection> getConnections(final Relationship relationship) {
-        readLock.lock();
-        try {
-            if (relationship.equals(Relationship.ANONYMOUS)) {
-                return Collections.unmodifiableSet(outgoingConnections);
-            }
-
-            throw new IllegalArgumentException("No relationship with name " + relationship.getName() + " exists for Funnels");
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public List<Connection> getIncomingConnections() {
-        readLock.lock();
-        try {
-            return new ArrayList<>(incomingConnections);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public Position getPosition() {
-        return position.get();
-    }
-
-    @Override
-    public void setPosition(Position position) {
-        this.position.set(position);
-    }
-
-    @Override
-    public String getName() {
-        return name.get();
-    }
-
-    /**
-     * Throws {@link UnsupportedOperationException}
-     *
-     * @param name
-     */
-    @Override
-    public void setName(final String name) {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public String getComments() {
-        return "";
-    }
-
-    @Override
-    public void setComments(final String comments) {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public ProcessGroup getProcessGroup() {
-        return processGroupRef.get();
-    }
-
-    @Override
-    public void setProcessGroup(final ProcessGroup group) {
-        processGroupRef.set(group);
-    }
-
-    @Override
-    public boolean isAutoTerminated(Relationship relationship) {
-        return false;
-    }
-
-    @Override
-    public boolean isRunning() {
-        return isRunning(this);
-    }
-
-    private boolean isRunning(final Connectable source) {
-        return getScheduledState() == ScheduledState.RUNNING;
-    }
-
-    @Override
-    public boolean isTriggerWhenEmpty() {
-        return false;
-    }
-
-    @Override
-    public ScheduledState getScheduledState() {
-        return scheduledState.get();
-    }
-
-    @Override
-    public boolean isLossTolerant() {
-        return lossTolerant.get();
-    }
-
-    @Override
-    public void setLossTolerant(final boolean lossTolerant) {
-        this.lossTolerant.set(lossTolerant);
-    }
-
-    @Override
-    public String toString() {
-        return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE).append("id", getIdentifier()).toString();
-    }
-
-    @Override
-    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
-        final ProcessSession session = sessionFactory.createSession();
-
-        try {
-            onTrigger(context, session);
-            session.commit();
-        } catch (final ProcessException e) {
-            session.rollback();
-            throw e;
-        } catch (final Throwable t) {
-            session.rollback();
-            throw new RuntimeException(t);
-        }
-    }
-
-    private void onTrigger(final ProcessContext context, final ProcessSession session) {
-        readLock.lock();
-        try {
-            Set<Relationship> available = session.getAvailableRelationships();
-            int transferred = 0;
-            while (!available.isEmpty()) {
-                final List<FlowFile> flowFiles = session.get(10);
-                if (flowFiles.isEmpty()) {
-                    break;
-                }
-
-                transferred += flowFiles.size();
-                session.transfer(flowFiles, Relationship.ANONYMOUS);
-                session.commit();
-                available = session.getAvailableRelationships();
-            }
-
-            if (transferred == 0) {
-                context.yield();
-            }
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    /**
-     * Has no effect
-     */
-    @Override
-    public void setMaxConcurrentTasks(int taskCount) {
-    }
-
-    @Override
-    public int getMaxConcurrentTasks() {
-        return 1;
-    }
-
-    @Override
-    public void setScheduledState(final ScheduledState scheduledState) {
-        this.scheduledState.set(scheduledState);
-    }
-
-    @Override
-    public ConnectableType getConnectableType() {
-        return ConnectableType.FUNNEL;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public Collection<ValidationResult> getValidationErrors() {
-        return Collections.EMPTY_LIST;
-    }
-
-    /**
-     * Updates the amount of time that this processor should avoid being
-     * scheduled when the processor calls
-     * {@link nifi.processor.ProcessContext#yield() ProcessContext.yield()}
-     *
-     * @param yieldPeriod
-     */
-    @Override
-    public void setYieldPeriod(final String yieldPeriod) {
-        final long yieldMillis = FormatUtils.getTimeDuration(requireNonNull(yieldPeriod), TimeUnit.MILLISECONDS);
-        if (yieldMillis < 0) {
-            throw new IllegalArgumentException("Yield duration must be positive");
-        }
-        this.yieldPeriod.set(yieldPeriod);
-    }
-
-    /**
-     * @param schedulingPeriod
-     */
-    @Override
-    public void setScheduldingPeriod(final String schedulingPeriod) {
-        final long schedulingNanos = FormatUtils.getTimeDuration(requireNonNull(schedulingPeriod), TimeUnit.NANOSECONDS);
-        if (schedulingNanos < 0) {
-            throw new IllegalArgumentException("Scheduling Period must be positive");
-        }
-
-        this.schedulingPeriod.set(schedulingPeriod);
-        this.schedulingNanos.set(Math.max(MINIMUM_SCHEDULING_NANOS, schedulingNanos));
-    }
-
-    @Override
-    public long getPenalizationPeriod(final TimeUnit timeUnit) {
-        return FormatUtils.getTimeDuration(getPenalizationPeriod(), timeUnit == null ? DEFAULT_TIME_UNIT : timeUnit);
-    }
-
-    @Override
-    public String getPenalizationPeriod() {
-        return penalizationPeriod.get();
-    }
-
-    /**
-     * Causes the processor not to be scheduled for some period of time. This
-     * duration can be obtained and set via the
-     * {@link #getYieldPeriod(TimeUnit)} and
-     * {@link #setYieldPeriod(long, TimeUnit)} methods.
-     */
-    @Override
-    public void yield() {
-        final long yieldMillis = getYieldPeriod(TimeUnit.MILLISECONDS);
-        yieldExpiration.set(Math.max(yieldExpiration.get(), System.currentTimeMillis() + yieldMillis));
-    }
-
-    @Override
-    public long getYieldExpiration() {
-        return yieldExpiration.get();
-    }
-
-    @Override
-    public String getSchedulingPeriod() {
-        return schedulingPeriod.get();
-    }
-
-    @Override
-    public void setPenalizationPeriod(final String penalizationPeriod) {
-        this.penalizationPeriod.set(penalizationPeriod);
-    }
-
-    @Override
-    public String getYieldPeriod() {
-        return yieldPeriod.get();
-    }
-
-    @Override
-    public long getYieldPeriod(final TimeUnit timeUnit) {
-        return FormatUtils.getTimeDuration(getYieldPeriod(), timeUnit == null ? DEFAULT_TIME_UNIT : timeUnit);
-    }
-
-    @Override
-    public long getSchedulingPeriod(final TimeUnit timeUnit) {
-        return timeUnit.convert(schedulingNanos.get(), TimeUnit.NANOSECONDS);
-    }
-
-    @Override
-    public boolean isSideEffectFree() {
-        return true;
-    }
-
-    @Override
-    public void verifyCanDelete(boolean ignoreConnections) throws IllegalStateException {
-        if (ignoreConnections) {
-            return;
-        }
-
-        readLock.lock();
-        try {
-            for (final Connection connection : outgoingConnections) {
-                connection.verifyCanDelete();
-            }
-
-            for (final Connection connection : incomingConnections) {
-                if (connection.getSource().equals(this)) {
-                    connection.verifyCanDelete();
-                } else {
-                    throw new IllegalStateException(this + " is the destination of another component");
-                }
-            }
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public void verifyCanDelete() {
-        verifyCanDelete(false);
-    }
-
-    @Override
-    public void verifyCanStart() {
-    }
-
-    @Override
-    public void verifyCanStop() {
-    }
-
-    @Override
-    public void verifyCanUpdate() {
-    }
-
-    @Override
-    public void verifyCanEnable() {
-    }
-
-    @Override
-    public void verifyCanDisable() {
-    }
-
-    @Override
-    public SchedulingStrategy getSchedulingStrategy() {
-        return SchedulingStrategy.TIMER_DRIVEN;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java
deleted file mode 100644
index df3c251..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java
+++ /dev/null
@@ -1,27 +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;
-
-import java.util.Map;
-
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.ValidationContext;
-
-public interface ValidationContextFactory {
-
-    ValidationContext newValidationContext(Map<PropertyDescriptor, String> properties, String annotationData);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/WorkerQueue.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/WorkerQueue.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/WorkerQueue.java
deleted file mode 100644
index 2f43600..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/WorkerQueue.java
+++ /dev/null
@@ -1,36 +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;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.connectable.Connectable;
-
-public interface WorkerQueue {
-
-    EventBasedWorker poll(long timeout, TimeUnit timeUnit);
-
-    void offer(Connectable worker);
-
-    void setClustered(boolean clustered);
-
-    void setPrimary(boolean primary);
-
-    void suspendWork(Connectable worker);
-
-    void resumeWork(Connectable worker);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/CommunicationsException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/CommunicationsException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/CommunicationsException.java
deleted file mode 100644
index 368ed1b..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/CommunicationsException.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.controller.exception;
-
-import java.io.IOException;
-
-public class CommunicationsException extends IOException {
-
-    private static final long serialVersionUID = 142343242323423L;
-
-    public CommunicationsException() {
-        super();
-    }
-
-    public CommunicationsException(final Throwable cause) {
-        super(cause);
-    }
-
-    public CommunicationsException(final String explanation) {
-        super(explanation);
-    }
-
-    public CommunicationsException(final String explanation, final Throwable cause) {
-        super(explanation, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceAlreadyExistsException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceAlreadyExistsException.java
deleted file mode 100644
index 0ff68b0..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceAlreadyExistsException.java
+++ /dev/null
@@ -1,30 +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.exception;
-
-public class ControllerServiceAlreadyExistsException extends RuntimeException {
-
-    private static final long serialVersionUID = -544424320587059277L;
-
-    /**
-     * Constructs a default exception
-     * @param id
-     */
-    public ControllerServiceAlreadyExistsException(final String id) {
-        super("A Controller Service already exists with ID " + id);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceNotFoundException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceNotFoundException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceNotFoundException.java
deleted file mode 100644
index 4cdbe54..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceNotFoundException.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.controller.exception;
-
-public class ControllerServiceNotFoundException extends RuntimeException {
-
-    private static final long serialVersionUID = -544424320587059277L;
-
-    /**
-     * Constructs a default exception
-     */
-    public ControllerServiceNotFoundException() {
-        super();
-    }
-
-    /**
-     * @param message
-     */
-    public ControllerServiceNotFoundException(String message) {
-        super(message);
-    }
-
-    /**
-     * @param cause
-     */
-    public ControllerServiceNotFoundException(Throwable cause) {
-        super(cause);
-    }
-
-    /**
-     * @param message
-     * @param cause
-     */
-    public ControllerServiceNotFoundException(String message, Throwable cause) {
-        super(message, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorInstantiationException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorInstantiationException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorInstantiationException.java
deleted file mode 100644
index c4aba44..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorInstantiationException.java
+++ /dev/null
@@ -1,27 +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.exception;
-
-public class ProcessorInstantiationException extends Exception {
-
-    private static final long serialVersionUID = 189273489L;
-
-    public ProcessorInstantiationException(final String className, final Throwable t) {
-        super(className, t);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorLifeCycleException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorLifeCycleException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorLifeCycleException.java
deleted file mode 100644
index 5acca16..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorLifeCycleException.java
+++ /dev/null
@@ -1,30 +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.exception;
-
-public class ProcessorLifeCycleException extends RuntimeException {
-
-    private static final long serialVersionUID = 8392341500511490941L;
-
-    public ProcessorLifeCycleException(final String message, final Throwable t) {
-        super(message, t);
-    }
-
-    public ProcessorLifeCycleException(final Throwable t) {
-        super(t);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/label/Label.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/label/Label.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/label/Label.java
deleted file mode 100644
index 97c44b5..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/label/Label.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.controller.label;
-
-import java.util.Map;
-
-import org.apache.nifi.connectable.Position;
-import org.apache.nifi.connectable.Size;
-import org.apache.nifi.groups.ProcessGroup;
-
-public interface Label {
-
-    String getIdentifier();
-
-    Position getPosition();
-
-    void setPosition(Position position);
-
-    Map<String, String> getStyle();
-
-    void setStyle(Map<String, String> style);
-
-    Size getSize();
-
-    void setSize(Size size);
-
-    ProcessGroup getProcessGroup();
-
-    void setProcessGroup(ProcessGroup group);
-
-    String getValue();
-
-    void setValue(String value);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskInstantiationException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskInstantiationException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskInstantiationException.java
deleted file mode 100644
index ced6ff9..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskInstantiationException.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.controller.reporting;
-
-public class ReportingTaskInstantiationException extends Exception {
-
-    private static final long serialVersionUID = 189234789237L;
-
-    public ReportingTaskInstantiationException(final String className, final Throwable t) {
-        super(className, t);
-    }
-
-    public ReportingTaskInstantiationException(final String message) {
-        super(message);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/ContentNotFoundException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/ContentNotFoundException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/ContentNotFoundException.java
deleted file mode 100644
index 6ce7ba6..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/ContentNotFoundException.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.controller.repository;
-
-import org.apache.nifi.controller.repository.claim.ContentClaim;
-
-/**
- *
- * @author none
- */
-public class ContentNotFoundException extends RuntimeException {
-
-    private static final long serialVersionUID = 19048239082L;
-    private final transient ContentClaim claim;
-
-    public ContentNotFoundException(final ContentClaim claim) {
-        super("Could not find content for " + claim);
-        this.claim = claim;
-    }
-
-    public ContentNotFoundException(final ContentClaim claim, final Throwable t) {
-        super("Could not find content for " + claim, t);
-        this.claim = claim;
-    }
-
-    public ContentNotFoundException(final ContentClaim claim, final String message) {
-        super("Could not find content for " + claim + ": " + message);
-        this.claim = claim;
-    }
-
-    public ContentClaim getMissingClaim() {
-        return claim;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/CounterRepository.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/CounterRepository.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/CounterRepository.java
deleted file mode 100644
index de231ed..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/CounterRepository.java
+++ /dev/null
@@ -1,34 +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;
-
-import java.util.List;
-
-import org.apache.nifi.controller.Counter;
-
-public interface CounterRepository {
-
-    void adjustCounter(String counterContext, String name, long delta);
-
-    Counter getCounter(String counterContext, String name);
-
-    List<Counter> getCounters();
-
-    List<Counter> getCounters(String counterContext);
-
-    Counter resetCounter(String identifier);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/FlowFileEvent.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/FlowFileEvent.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/FlowFileEvent.java
deleted file mode 100644
index f07a530..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/FlowFileEvent.java
+++ /dev/null
@@ -1,54 +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;
-
-public interface FlowFileEvent {
-
-    String getComponentIdentifier();
-
-    int getFlowFilesIn();
-
-    int getFlowFilesOut();
-
-    int getFlowFilesRemoved();
-
-    long getContentSizeIn();
-
-    long getContentSizeOut();
-
-    long getContentSizeRemoved();
-
-    long getBytesRead();
-
-    long getBytesWritten();
-
-    long getProcessingNanoseconds();
-
-    long getAverageLineageMillis();
-
-    long getAggregateLineageMillis();
-
-    int getFlowFilesReceived();
-
-    long getBytesReceived();
-
-    int getFlowFilesSent();
-
-    long getBytesSent();
-
-    int getInvocations();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/FlowFileEventRepository.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/FlowFileEventRepository.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/FlowFileEventRepository.java
deleted file mode 100644
index 2eb3caf..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/FlowFileEventRepository.java
+++ /dev/null
@@ -1,50 +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;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-/**
- *
- * @author none
- */
-public interface FlowFileEventRepository extends Closeable {
-
-    /**
-     * Updates the repository to include a new FlowFile processing event
-     *
-     * @param event
-     * @throws java.io.IOException
-     */
-    void updateRepository(FlowFileEvent event) throws IOException;
-
-    /**
-     * Returns a report of processing activity since the given time
-     * @param sinceEpochMillis
-     * @return 
-     */
-    RepositoryStatusReport reportTransferEvents(long sinceEpochMillis);
-
-    /**
-     * Causes any flow file events of the given entry age in epoch milliseconds
-     * or older to be purged from the repository
-     *
-     * @param cutoffEpochMilliseconds
-     */
-    void purgeTransferEvents(long cutoffEpochMilliseconds);
-}


[34/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/maven-plugins/nar-maven-plugin/src/main/java/nifi/NarMojo.java
----------------------------------------------------------------------
diff --git a/maven-plugins/nar-maven-plugin/src/main/java/nifi/NarMojo.java b/maven-plugins/nar-maven-plugin/src/main/java/nifi/NarMojo.java
new file mode 100644
index 0000000..9b70ec0
--- /dev/null
+++ b/maven-plugins/nar-maven-plugin/src/main/java/nifi/NarMojo.java
@@ -0,0 +1,613 @@
+/*
+ * 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 nifi;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.maven.archiver.MavenArchiveConfiguration;
+import org.apache.maven.archiver.MavenArchiver;
+import org.apache.maven.artifact.Artifact;
+import org.apache.maven.artifact.DependencyResolutionRequiredException;
+import org.apache.maven.artifact.factory.ArtifactFactory;
+import org.apache.maven.artifact.installer.ArtifactInstaller;
+import org.apache.maven.artifact.metadata.ArtifactMetadataSource;
+import org.apache.maven.artifact.repository.ArtifactRepository;
+import org.apache.maven.artifact.repository.ArtifactRepositoryFactory;
+import org.apache.maven.artifact.resolver.ArtifactCollector;
+import org.apache.maven.artifact.resolver.ArtifactNotFoundException;
+import org.apache.maven.artifact.resolver.ArtifactResolutionException;
+import org.apache.maven.artifact.resolver.ArtifactResolver;
+import org.apache.maven.plugin.AbstractMojo;
+import org.apache.maven.plugin.MojoExecutionException;
+import org.apache.maven.plugin.MojoFailureException;
+import org.apache.maven.plugin.dependency.utils.DependencyStatusSets;
+import org.apache.maven.plugin.dependency.utils.DependencyUtil;
+import org.apache.maven.plugin.dependency.utils.filters.DestFileFilter;
+import org.apache.maven.plugin.dependency.utils.resolvers.ArtifactsResolver;
+import org.apache.maven.plugin.dependency.utils.resolvers.DefaultArtifactsResolver;
+import org.apache.maven.plugin.dependency.utils.translators.ArtifactTranslator;
+import org.apache.maven.plugin.dependency.utils.translators.ClassifierTypeTranslator;
+import org.apache.maven.plugins.annotations.LifecyclePhase;
+import org.apache.maven.plugins.annotations.Mojo;
+import org.apache.maven.plugins.annotations.Parameter;
+import org.apache.maven.plugins.annotations.ResolutionScope;
+import org.apache.maven.project.MavenProject;
+import org.apache.maven.execution.MavenSession;
+import org.apache.maven.plugins.annotations.Component;
+import org.apache.maven.project.MavenProjectHelper;
+import org.apache.maven.shared.artifact.filter.collection.ArtifactFilterException;
+import org.apache.maven.shared.artifact.filter.collection.ArtifactIdFilter;
+import org.apache.maven.shared.artifact.filter.collection.ArtifactsFilter;
+import org.apache.maven.shared.artifact.filter.collection.ClassifierFilter;
+import org.apache.maven.shared.artifact.filter.collection.FilterArtifacts;
+import org.apache.maven.shared.artifact.filter.collection.GroupIdFilter;
+import org.apache.maven.shared.artifact.filter.collection.ScopeFilter;
+import org.apache.maven.shared.artifact.filter.collection.ProjectTransitivityFilter;
+import org.apache.maven.shared.artifact.filter.collection.TypeFilter;
+import org.codehaus.plexus.archiver.ArchiverException;
+import org.codehaus.plexus.archiver.jar.JarArchiver;
+import org.codehaus.plexus.archiver.jar.ManifestException;
+import org.codehaus.plexus.archiver.manager.ArchiverManager;
+import org.codehaus.plexus.util.FileUtils;
+import org.codehaus.plexus.util.StringUtils;
+
+/**
+ * Packages the current project as an Apache NiFi Archive (NAR).
+ *
+ * The following code is derived from maven-dependencies-plugin and
+ * maven-jar-plugin. The functionality of CopyDependenciesMojo and JarMojo was
+ * simplified to the use case of NarMojo.
+ *
+ */
+@Mojo(name = "nar", defaultPhase = LifecyclePhase.PACKAGE, threadSafe = false, requiresDependencyResolution = ResolutionScope.RUNTIME)
+public class NarMojo extends AbstractMojo {
+
+    private static final String[] DEFAULT_EXCLUDES = new String[]{"**/package.html"};
+    private static final String[] DEFAULT_INCLUDES = new String[]{"**/**"};
+
+    /**
+     * POM
+     *
+     */
+    @Parameter(defaultValue = "${project}", readonly = true, required = true)
+    protected MavenProject project;
+
+    @Parameter(defaultValue = "${session}", readonly = true, required = true)
+    protected MavenSession session;
+
+    /**
+     * List of files to include. Specified as fileset patterns.
+     */
+    @Parameter(property = "includes")
+    protected String[] includes;
+    /**
+     * List of files to exclude. Specified as fileset patterns.
+     */
+    @Parameter(property = "excludes")
+    protected String[] excludes;
+    /**
+     * Name of the generated NAR.
+     *
+     */
+    @Parameter(alias = "narName", property = "nar.finalName", defaultValue = "${project.build.finalName}", required = true)
+    protected String finalName;
+
+    /**
+     * The Jar archiver.
+     *
+     * \@\component role="org.codehaus.plexus.archiver.Archiver" roleHint="jar"
+     */
+    @Component(role = org.codehaus.plexus.archiver.Archiver.class, hint = "jar")
+    private JarArchiver jarArchiver;
+    /**
+     * The archive configuration to use.
+     *
+     * See <a
+     * href="http://maven.apache.org/shared/maven-archiver/index.html">the
+     * documentation for Maven Archiver</a>.
+     *
+     */
+    @Parameter(property = "archive")
+    protected final MavenArchiveConfiguration archive = new MavenArchiveConfiguration();
+    /**
+     * Path to the default MANIFEST file to use. It will be used if
+     * <code>useDefaultManifestFile</code> is set to <code>true</code>.
+     *
+     */
+    @Parameter(property = "defaultManifestFiles", defaultValue = "${project.build.outputDirectory}/META-INF/MANIFEST.MF", readonly = true, required = true)
+    protected File defaultManifestFile;
+
+    /**
+     * Set this to <code>true</code> to enable the use of the
+     * <code>defaultManifestFile</code>.
+     *
+     * @since 2.2
+     */
+    @Parameter(property = "nar.useDefaultManifestFile", defaultValue = "false")
+    protected boolean useDefaultManifestFile;
+
+    @Component
+    protected MavenProjectHelper projectHelper;
+
+    /**
+     * Whether creating the archive should be forced.
+     *
+     */
+    @Parameter(property = "nar.forceCreation", defaultValue = "false")
+    protected boolean forceCreation;
+
+    /**
+     * Classifier to add to the artifact generated. If given, the artifact will
+     * be an attachment instead.
+     *
+     */
+    @Parameter(property = "classifier")
+    protected String classifier;
+
+    @Component
+    protected ArtifactInstaller installer;
+
+    @Component
+    protected ArtifactRepositoryFactory repositoryFactory;
+
+    /**
+     * This only applies if the classifier parameter is used.
+     *
+     */
+    @Parameter(property = "mdep.failOnMissingClassifierArtifact", defaultValue = "true", required = false)
+    protected boolean failOnMissingClassifierArtifact = true;
+
+    /**
+     * Comma Separated list of Types to include. Empty String indicates include
+     * everything (default).
+     *
+     */
+    @Parameter(property = "includeTypes", required = false)
+    protected String includeTypes;
+
+    /**
+     * Comma Separated list of Types to exclude. Empty String indicates don't
+     * exclude anything (default).
+     *
+     */
+    @Parameter(property = "excludeTypes", required = false)
+    protected String excludeTypes;
+
+    /**
+     * Scope to include. An Empty string indicates all scopes (default).
+     *
+     */
+    @Parameter(property = "includeScope", required = false)
+    protected String includeScope;
+
+    /**
+     * Scope to exclude. An Empty string indicates no scopes (default).
+     *
+     */
+    @Parameter(property = "excludeScope", required = false)
+    protected String excludeScope;
+
+    /**
+     * Comma Separated list of Classifiers to include. Empty String indicates
+     * include everything (default).
+     *
+     */
+    @Parameter(property = "includeClassifiers", required = false)
+    protected String includeClassifiers;
+
+    /**
+     * Comma Separated list of Classifiers to exclude. Empty String indicates
+     * don't exclude anything (default).
+     *
+     */
+    @Parameter(property = "excludeClassifiers", required = false)
+    protected String excludeClassifiers;
+
+    /**
+     * Specify classifier to look for. Example: sources
+     *
+     */
+    @Parameter(property = "classifier", required = false)
+    protected String copyDepClassifier;
+
+    /**
+     * Specify type to look for when constructing artifact based on classifier.
+     * Example: java-source,jar,war, nar
+     *
+     */
+    @Parameter(property = "type", required = false, defaultValue = "nar")
+    protected String type;
+
+    /**
+     * Comma separated list of Artifact names too exclude.
+     *
+     */
+    @Parameter(property = "excludeArtifacts", required = false)
+    protected String excludeArtifactIds;
+
+    /**
+     * Comma separated list of Artifact names to include.
+     *
+     */
+    @Parameter(property = "includeArtifacts", required = false)
+    protected String includeArtifactIds;
+
+    /**
+     * Comma separated list of GroupId Names to exclude.
+     *
+     */
+    @Parameter(property = "excludeArtifacts", required = false)
+    protected String excludeGroupIds;
+
+    /**
+     * Comma separated list of GroupIds to include.
+     *
+     */
+    @Parameter(property = "includeGroupIds", required = false)
+    protected String includeGroupIds;
+
+    /**
+     * Directory to store flag files
+     *
+     */
+    @Parameter(property = "markersDirectory", required = false, defaultValue = "${project.build.directory}/dependency-maven-plugin-markers")
+    protected File markersDirectory;
+
+    /**
+     * Overwrite release artifacts
+     *
+     */
+    @Parameter(property = "overWriteReleases", required = false)
+    protected boolean overWriteReleases;
+
+    /**
+     * Overwrite snapshot artifacts
+     *
+     */
+    @Parameter(property = "overWriteSnapshots", required = false)
+    protected boolean overWriteSnapshots;
+
+    /**
+     * Overwrite artifacts that don't exist or are older than the source.
+     *
+     */
+    @Parameter(property = "overWriteIfNewer", required = false, defaultValue = "true")
+    protected boolean overWriteIfNewer;
+    
+    @Parameter( property = "projectBuildDirectory", required = false, defaultValue = "${project.build.directory}")
+    protected File projectBuildDirectory;
+
+    /**
+     * Used to look up Artifacts in the remote repository.
+     */
+    @Component
+    protected ArtifactFactory factory;
+
+    /**
+     * Used to look up Artifacts in the remote repository.
+     *
+     */
+    @Component
+    protected ArtifactResolver resolver;
+
+    /**
+     * Artifact collector, needed to resolve dependencies.
+     *
+     */
+    @Component(role = org.apache.maven.artifact.resolver.ArtifactCollector.class)
+    protected ArtifactCollector artifactCollector;
+
+    @Component(role = org.apache.maven.artifact.metadata.ArtifactMetadataSource.class)
+    protected ArtifactMetadataSource artifactMetadataSource;
+
+    /**
+     * Location of the local repository.
+     *
+     */
+    @Parameter(property = "localRepository", required = true, readonly = true)
+    protected ArtifactRepository local;
+
+    /**
+     * List of Remote Repositories used by the resolver
+     *
+     */
+    @Parameter(property = "project.remoteArtifactRepositories", required = true, readonly = true)
+    protected List remoteRepos;
+
+    /**
+     * To look up Archiver/UnArchiver implementations
+     *
+     */
+    @Component
+    protected ArchiverManager archiverManager;
+
+    /**
+     * Contains the full list of projects in the reactor.
+     *
+     */
+    @Parameter(property = "reactorProjects", required = true, readonly = true)
+    protected List reactorProjects;
+
+    /**
+     * If the plugin should be silent.
+     *
+     */
+    @Parameter(property = "silent", required = false, defaultValue = "false")
+    public boolean silent;
+
+    /**
+     * Output absolute filename for resolved artifacts
+     *
+     */
+    @Parameter(property = "outputAbsoluteArtifactFilename", defaultValue = "false", required = false)
+    protected boolean outputAbsoluteArtifactFilename;
+
+    @Override
+    public void execute() throws MojoExecutionException, MojoFailureException {
+        copyDependencies();
+        makeNar();
+    }
+
+    private void copyDependencies() throws MojoExecutionException {
+        DependencyStatusSets dss = getDependencySets(this.failOnMissingClassifierArtifact);
+        Set artifacts = dss.getResolvedDependencies();
+
+        for (Object artifactObj : artifacts) {
+            copyArtifact((Artifact) artifactObj);
+        }
+
+        artifacts = dss.getSkippedDependencies();
+        for (Object artifactOjb : artifacts) {
+            Artifact artifact = (Artifact) artifactOjb;
+            getLog().info(artifact.getFile().getName() + " already exists in destination.");
+        }
+    }
+
+    protected void copyArtifact(Artifact artifact) throws MojoExecutionException {
+        String destFileName = DependencyUtil.getFormattedFileName(artifact, false);
+        final File destDir = DependencyUtil.getFormattedOutputDirectory(false, false, false, false, false, getDependenciesDirectory(), artifact);
+        final File destFile = new File(destDir, destFileName);
+        copyFile(artifact.getFile(), destFile);
+    }
+
+    protected Artifact getResolvedPomArtifact(Artifact artifact) {
+        Artifact pomArtifact = this.factory.createArtifact(artifact.getGroupId(), artifact.getArtifactId(), artifact.getVersion(), "", "pom");
+        // Resolve the pom artifact using repos
+        try {
+            this.resolver.resolve(pomArtifact, this.remoteRepos, this.local);
+        } catch (ArtifactResolutionException | ArtifactNotFoundException e) {
+            getLog().info(e.getMessage());
+        }
+        return pomArtifact;
+    }
+
+    protected ArtifactsFilter getMarkedArtifactFilter() {
+        return new DestFileFilter(this.overWriteReleases, this.overWriteSnapshots, this.overWriteIfNewer, false, false, false, false, false, getDependenciesDirectory());
+    }
+
+    protected DependencyStatusSets getDependencySets(boolean stopOnFailure) throws MojoExecutionException {
+        // add filters in well known order, least specific to most specific
+        FilterArtifacts filter = new FilterArtifacts();
+
+        filter.addFilter(new ProjectTransitivityFilter(project.getDependencyArtifacts(), false));
+        filter.addFilter(new ScopeFilter(this.includeScope, this.excludeScope));
+        filter.addFilter(new TypeFilter(this.includeTypes, this.excludeTypes));
+        filter.addFilter(new ClassifierFilter(this.includeClassifiers, this.excludeClassifiers));
+        filter.addFilter(new GroupIdFilter(this.includeGroupIds, this.excludeGroupIds));
+        filter.addFilter(new ArtifactIdFilter(this.includeArtifactIds, this.excludeArtifactIds));
+
+        // explicitly filter our nar dependencies
+        filter.addFilter(new TypeFilter("", "nar"));
+
+        // start with all artifacts.
+        Set artifacts = project.getArtifacts();
+
+        // perform filtering
+        try {
+            artifacts = filter.filter(artifacts);
+        } catch (ArtifactFilterException e) {
+            throw new MojoExecutionException(e.getMessage(), e);
+        }
+
+        // transform artifacts if classifier is set
+        final DependencyStatusSets status;
+        if (StringUtils.isNotEmpty(copyDepClassifier)) {
+            status = getClassifierTranslatedDependencies(artifacts, stopOnFailure);
+        } else {
+            status = filterMarkedDependencies(artifacts);
+        }
+
+        return status;
+    }
+
+    protected DependencyStatusSets getClassifierTranslatedDependencies(Set artifacts, boolean stopOnFailure) throws MojoExecutionException {
+        Set unResolvedArtifacts = new HashSet();
+        Set resolvedArtifacts = artifacts;
+        DependencyStatusSets status = new DependencyStatusSets();
+
+        // possibly translate artifacts into a new set of artifacts based on the
+        // classifier and type
+        // if this did something, we need to resolve the new artifacts
+        if (StringUtils.isNotEmpty(copyDepClassifier)) {
+            ArtifactTranslator translator = new ClassifierTypeTranslator(this.copyDepClassifier, this.type, this.factory);
+            artifacts = translator.translate(artifacts, getLog());
+
+            status = filterMarkedDependencies(artifacts);
+
+            // the unskipped artifacts are in the resolved set.
+            artifacts = status.getResolvedDependencies();
+
+            // resolve the rest of the artifacts
+            ArtifactsResolver artifactsResolver = new DefaultArtifactsResolver(this.resolver, this.local,
+                    this.remoteRepos, stopOnFailure);
+            resolvedArtifacts = artifactsResolver.resolve(artifacts, getLog());
+
+            // calculate the artifacts not resolved.
+            unResolvedArtifacts.addAll(artifacts);
+            unResolvedArtifacts.removeAll(resolvedArtifacts);
+        }
+
+        // return a bean of all 3 sets.
+        status.setResolvedDependencies(resolvedArtifacts);
+        status.setUnResolvedDependencies(unResolvedArtifacts);
+
+        return status;
+    }
+
+    protected DependencyStatusSets filterMarkedDependencies(Set artifacts) throws MojoExecutionException {
+        // remove files that have markers already
+        FilterArtifacts filter = new FilterArtifacts();
+        filter.clearFilters();
+        filter.addFilter(getMarkedArtifactFilter());
+
+        Set unMarkedArtifacts;
+        try {
+            unMarkedArtifacts = filter.filter(artifacts);
+        } catch (ArtifactFilterException e) {
+            throw new MojoExecutionException(e.getMessage(), e);
+        }
+
+        // calculate the skipped artifacts
+        Set skippedArtifacts = new HashSet();
+        skippedArtifacts.addAll(artifacts);
+        skippedArtifacts.removeAll(unMarkedArtifacts);
+
+        return new DependencyStatusSets(unMarkedArtifacts, null, skippedArtifacts);
+    }
+
+    protected void copyFile(File artifact, File destFile) throws MojoExecutionException {
+        try {
+            getLog().info("Copying " + (this.outputAbsoluteArtifactFilename ? artifact.getAbsolutePath() : artifact.getName()) + " to " + destFile);
+            FileUtils.copyFile(artifact, destFile);
+        } catch (Exception e) {
+            throw new MojoExecutionException("Error copying artifact from " + artifact + " to " + destFile, e);
+        }
+    }
+
+    private File getClassesDirectory() {
+        final File outputDirectory = projectBuildDirectory;
+        return new File(outputDirectory, "classes");
+    }
+
+    private File getDependenciesDirectory() {
+        return new File(getClassesDirectory(), "META-INF/bundled-dependencies");
+    }
+
+    private void makeNar() throws MojoExecutionException {
+        File narFile = createArchive();
+
+        if (classifier != null) {
+            projectHelper.attachArtifact(project, "nar", classifier, narFile);
+        } else {
+            project.getArtifact().setFile(narFile);
+        }
+    }
+
+    public File createArchive() throws MojoExecutionException {
+        final File outputDirectory = projectBuildDirectory;
+        File narFile = getNarFile(outputDirectory, finalName, classifier);
+        MavenArchiver archiver = new MavenArchiver();
+        archiver.setArchiver(jarArchiver);
+        archiver.setOutputFile(narFile);
+        archive.setForced(forceCreation);
+
+        try {
+            File contentDirectory = getClassesDirectory();
+            if (!contentDirectory.exists()) {
+                getLog().warn("NAR will be empty - no content was marked for inclusion!");
+            } else {
+                archiver.getArchiver().addDirectory(contentDirectory, getIncludes(), getExcludes());
+            }
+
+            File existingManifest = defaultManifestFile;
+            if (useDefaultManifestFile && existingManifest.exists() && archive.getManifestFile() == null) {
+                getLog().info("Adding existing MANIFEST to archive. Found under: " + existingManifest.getPath());
+                archive.setManifestFile(existingManifest);
+            }
+
+            // automatically add the artifact id to the manifest
+            archive.addManifestEntry("Nar-Id", project.getArtifactId());
+
+            // look for a nar dependency
+            String narDependency = getNarDependency();
+            if (narDependency != null) {
+                archive.addManifestEntry("Nar-Dependency-Id", narDependency);
+            }
+
+            archiver.createArchive(session, project, archive);
+            return narFile;
+        } catch (ArchiverException | MojoExecutionException | ManifestException | IOException | DependencyResolutionRequiredException e) {
+            throw new MojoExecutionException("Error assembling NAR", e);
+        }
+    }
+
+    private String[] getIncludes() {
+        if (includes != null && includes.length > 0) {
+            return includes;
+        }
+        return DEFAULT_INCLUDES;
+    }
+
+    private String[] getExcludes() {
+        if (excludes != null && excludes.length > 0) {
+            return excludes;
+        }
+        return DEFAULT_EXCLUDES;
+    }
+
+    protected File getNarFile(File basedir, String finalName, String classifier) {
+        if (classifier == null) {
+            classifier = "";
+        } else if (classifier.trim().length() > 0 && !classifier.startsWith("-")) {
+            classifier = "-" + classifier;
+        }
+
+        return new File(basedir, finalName + classifier + ".nar");
+    }
+
+    private String getNarDependency() throws MojoExecutionException {
+        String narDependency = null;
+
+        // get nar dependencies
+        FilterArtifacts filter = new FilterArtifacts();
+        filter.addFilter(new TypeFilter("nar", ""));
+
+        // start with all artifacts.
+        Set artifacts = project.getArtifacts();
+
+        // perform filtering
+        try {
+            artifacts = filter.filter(artifacts);
+        } catch (ArtifactFilterException e) {
+            throw new MojoExecutionException(e.getMessage(), e);
+        }
+
+        // ensure there is a single nar dependency
+        if (artifacts.size() > 1) {
+            throw new MojoExecutionException("Each NAR represents a ClassLoader. A NAR dependency allows that NAR's ClassLoader to be "
+                    + "used as the parent of this NAR's ClassLoader. As a result, only a single NAR dependency is allowed.");
+        } else if (artifacts.size() == 1) {
+            final Artifact artifact = (Artifact) artifacts.iterator().next();
+            narDependency = artifact.getArtifactId();
+        }
+
+        return narDependency;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/maven-plugins/nar-maven-plugin/src/main/resources/META-INF/plexus/components.xml
----------------------------------------------------------------------
diff --git a/maven-plugins/nar-maven-plugin/src/main/resources/META-INF/plexus/components.xml b/maven-plugins/nar-maven-plugin/src/main/resources/META-INF/plexus/components.xml
new file mode 100644
index 0000000..0680d18
--- /dev/null
+++ b/maven-plugins/nar-maven-plugin/src/main/resources/META-INF/plexus/components.xml
@@ -0,0 +1,52 @@
+<?xml version="1.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.
+-->
+<component-set>
+    <components>
+        <component>
+            <role>org.apache.maven.lifecycle.mapping.LifecycleMapping</role>
+            <role-hint>nar</role-hint>
+            <implementation>org.apache.maven.lifecycle.mapping.DefaultLifecycleMapping</implementation>
+            <configuration>
+                <lifecycles>
+                    <lifecycle>
+                        <id>default</id>
+                        <phases>
+                            <process-resources>org.apache.maven.plugins:maven-resources-plugin:resources</process-resources>
+                            <compile>org.apache.maven.plugins:maven-compiler-plugin:compile</compile>
+                            <process-test-resources>org.apache.maven.plugins:maven-resources-plugin:testResources</process-test-resources>
+                            <test-compile>org.apache.maven.plugins:maven-compiler-plugin:testCompile</test-compile>
+                            <test>org.apache.maven.plugins:maven-surefire-plugin:test</test>
+                            <package>org.apache.nifi:nar-maven-plugin:nar</package>
+                            <install>org.apache.maven.plugins:maven-install-plugin:install</install>
+                            <deploy>org.apache.maven.plugins:maven-deploy-plugin:deploy</deploy>
+                        </phases>
+                    </lifecycle>
+                </lifecycles>
+            </configuration>
+        </component>
+        <component>
+            <role>org.apache.maven.artifact.handler.ArtifactHandler</role>
+            <role-hint>nar</role-hint>
+            <implementation>org.apache.maven.artifact.handler.DefaultArtifactHandler</implementation>
+            <configuration>
+                <type>nar</type>
+                <language>java</language>
+                <addedToClasspath>false</addedToClasspath>
+                <includesDependencies>true</includesDependencies>
+            </configuration>
+        </component>
+    </components>
+</component-set>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/maven-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/maven-plugins/pom.xml b/maven-plugins/pom.xml
new file mode 100644
index 0000000..31d2bdc
--- /dev/null
+++ b/maven-plugins/pom.xml
@@ -0,0 +1,346 @@
+<?xml version="1.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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache</groupId>
+        <artifactId>apache</artifactId>
+        <version>16</version>
+        <relativePath/>
+    </parent>
+    <groupId>org.apache.nifi</groupId>
+    <artifactId>maven-plugins</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <packaging>pom</packaging>
+    <name>Apache NiFi Maven Plugins</name>
+    <description>Apache NiFi Maven Plugins. It is currently a part of the Apache Incubator.</description>
+    <url>http://nifi.incubator.apache.org/maven-site/</url>
+    <organization>
+        <name>Apache NiFi (incubating) Project</name>
+        <url>http://nifi.incubating.apache.org/</url>
+    </organization>
+    <licenses>
+        <license>
+            <name>Apache License, Version 2.0</name>
+            <url>http://www.apache.org/licenses/LICENSE-2.0</url>
+        </license>
+    </licenses>
+    <mailingLists>
+        <mailingList>
+            <name>Dev</name>
+            <subscribe>dev-subscribe@nifi.incubator.apache.org</subscribe>
+            <unsubscribe>dev-unsubscribe@nifi.incubator.apache.org</unsubscribe>
+            <post>dev@nifi.incubator.apache.org</post>
+            <archive>http://mail-archives.apache.org/mod_mbox/incubator-nifi-dev</archive>
+        </mailingList>
+        <mailingList>
+            <name>Commits</name>
+            <subscribe>commits-subscribe@nifi.incubator.apache.org</subscribe>
+            <unsubscribe>commits-unsubscribe@nifi.incubator.apache.org</unsubscribe>
+            <post>commits@nifi.incubator.apache.org</post>
+            <archive>http://mail-archives.apache.org/mod_mbox/incubator-nifi-commits</archive>
+        </mailingList>
+    </mailingLists>
+    <prerequisites>
+        <maven>${maven.min-version}</maven>
+    </prerequisites>
+    <modules>
+        <module>nar-maven-plugin</module>
+    </modules>
+    <scm>
+        <connection>scm:git:git://git.apache.org/incubator-nifi.git</connection>
+        <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-nifi.git</developerConnection>
+        <url>https://git-wip-us.apache.org/repos/asf?p=incubator-nifi.git</url>
+    </scm>
+    <issueManagement>
+        <system>JIRA</system>
+        <url>https://issues.apache.org/jira/browse/NIFI</url>
+    </issueManagement>
+    <properties>
+        <maven.compiler.source>1.7</maven.compiler.source>
+        <maven.compiler.target>1.7</maven.compiler.target>
+        <maven.version>3.0.5</maven.version>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+        <sealJars>false</sealJars>
+    </properties>
+    <build>
+        <pluginManagement>
+            <plugins>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-compiler-plugin</artifactId>
+                    <version>3.2</version>
+                    <configuration>
+                        <fork>true</fork>
+                        <optimize>true</optimize>
+                        <showDeprecation>true</showDeprecation>
+                        <showWarnings>true</showWarnings>
+                    </configuration>
+                </plugin>
+                <plugin>
+                    <artifactId>maven-war-plugin</artifactId>
+                    <version>2.5</version>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-plugin-plugin</artifactId>
+                    <version>3.3</version>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-dependency-plugin</artifactId>
+                    <version>2.9</version>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-resources-plugin</artifactId>
+                    <version>2.7</version>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-surefire-plugin</artifactId>
+                    <version>2.18</version>
+                    <configuration>
+                        <argLine>-Xmx1G</argLine>
+                    </configuration>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-assembly-plugin</artifactId>
+                    <version>2.5.2</version>
+                    <configuration>
+                        <attach>false</attach>
+                        <tarLongFileMode>gnu</tarLongFileMode>
+                    </configuration>
+                </plugin>
+                <plugin>
+                    <groupId>org.codehaus.mojo</groupId>
+                    <artifactId>jaxb2-maven-plugin</artifactId>
+                    <version>1.6</version>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-source-plugin</artifactId>
+                    <version>2.4</version>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-site-plugin</artifactId>
+                    <version>3.4</version>
+                </plugin>
+                <plugin>
+                    <groupId>org.codehaus.mojo</groupId>
+                    <artifactId>exec-maven-plugin</artifactId>
+                    <version>1.3.2</version>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-javadoc-plugin</artifactId>
+                    <version>2.10.1</version>
+                    <configuration>
+                        <failOnError>false</failOnError>
+                        <quiet>true</quiet>
+                        <show>private</show>
+                        <encoding>UTF-8</encoding>
+                        <quiet>true</quiet>
+                        <javadocVersion>1.7</javadocVersion>
+                        <additionalJOption>-J-Xmx512m</additionalJOption>
+                    </configuration>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-release-plugin</artifactId>
+                    <version>2.5.1</version>
+                    <executions>
+                        <execution>
+                            <id>default</id>
+                            <goals>
+                                <goal>prepare</goal>
+                                <goal>perform</goal>
+                            </goals>
+                            <configuration>
+                                <pomFileName>platform/pom.xml</pomFileName>
+                                <arguments>-P apache-release,check-licenses</arguments>
+                                <autoVersionSubmodules>true</autoVersionSubmodules>
+                                <releaseProfiles>apache-release</releaseProfiles>
+                                <goals>deploy</goals>
+                                <tagNameFormat>@{project.artifactId}-@{project.version}</tagNameFormat>
+                                <pushChanges>false</pushChanges>
+                                <localCheckout>true</localCheckout>
+                            </configuration>
+                        </execution>
+                    </executions>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-jar-plugin</artifactId>
+                    <configuration>
+                        <archive>
+                            <manifestEntries>
+                                <Sealed>${sealJars}</Sealed>
+                                <Implementation-Build>${mvngit.commit.id}</Implementation-Build>
+                            </manifestEntries>
+                        </archive>
+                    </configuration>
+                </plugin>
+            </plugins>
+        </pluginManagement>       
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-enforcer-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>enforce-maven</id>
+                        <goals>
+                            <goal>enforce</goal>
+                        </goals>
+                        <configuration>
+                            <rules>
+                                <requireSameVersions>
+                                    <plugins>
+                                        <plugin>org.apache.maven.plugins:maven-surefire-plugin</plugin>
+                                        <plugin>org.apache.maven.plugins:maven-failsafe-plugin</plugin>
+                                        <plugin>org.apache.maven.plugins:maven-surefire-report-plugin</plugin>
+                                    </plugins>
+                                </requireSameVersions>
+                                <requireMavenVersion>
+                                    <version>${maven.version}</version>
+                                </requireMavenVersion>
+                            </rules>    
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+    <dependencyManagement>
+        <dependencies>
+            <dependency>
+                <groupId>org.apache.maven</groupId>
+                <artifactId>maven-plugin-api</artifactId>
+                <version>2.0.11</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <version>2.9</version>
+                <type>maven-plugin</type>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <version>2.5</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.maven.plugin-tools</groupId>
+                <artifactId>maven-plugin-annotations</artifactId>
+                <version>3.3</version>
+            </dependency>                        
+        </dependencies>
+    </dependencyManagement>
+    <profiles>
+        <profile>
+            <id>apache-release</id>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-assembly-plugin</artifactId>
+                        <dependencies>
+                            <dependency>
+                                <groupId>org.apache.apache.resources</groupId>
+                                <artifactId>apache-source-release-assembly-descriptor</artifactId>
+                                <version>1.0.4</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>source-release-assembly</id>
+                                <goals>
+                                    <goal>single</goal>
+                                </goals>
+                                <phase>validate</phase>
+                                <configuration>
+                                    <runOnlyAtExecutionRoot>true</runOnlyAtExecutionRoot>
+                                    <finalName>nifi-${project.artifactId}-${project.version}</finalName>
+                                    <descriptorRefs>
+                                        <descriptorRef>source-release-zip-tar</descriptorRef>
+                                    </descriptorRefs>
+                                    <tarLongFileFormat>gnu</tarLongFileFormat>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>exec-maven-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>rename-source-release-assembly</id>
+                                <goals>
+                                    <goal>exec</goal>
+                                </goals>
+                                <phase>validate</phase>
+                                <configuration>
+                                    <executable>mv</executable>
+                                    <workingDirectory>${project.build.directory}</workingDirectory>
+                                    <commandlineArgs>-n nifi-${project.artifactId}-${project.version}-source-release.tar.gz nifi-${project.artifactId}-${project.version}-src.tar.gz</commandlineArgs>
+                                    <successCodes>
+                                        <successCode>0</successCode>
+                                        <successCode>1</successCode>
+                                    </successCodes>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+        <profile>
+            <!-- Seal jars and skip tests when the
+            apache-release profile is activated. -->
+            <id>seal-jars</id>
+            <properties>
+                <sealJars>true</sealJars>
+                <skipTests>true</skipTests>
+            </properties>
+        </profile>
+        <profile>
+            <!-- Automatically check for licenses.
+            Activate with -P check-licenses -->
+            <id>check-licenses</id>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.rat</groupId>
+                        <artifactId>apache-rat-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>check</goal>
+                                </goals>
+                                <phase>verify</phase>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/pom.xml b/nar-bundles/execute-script-bundle/execute-script-processors/pom.xml
deleted file mode 100644
index a5d3d11..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/pom.xml
+++ /dev/null
@@ -1,81 +0,0 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-    <!--
-      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.
-    -->
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>execute-script-bundle</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-    <artifactId>execute-script-processors</artifactId>
-    <description>NiFi Processors to Run Scripts</description>
-    <name>NiFi Script Execution Processors</name>
-    <dependencies>
-
-        <dependency>
-            <groupId>org.jruby</groupId>
-            <artifactId>jruby</artifactId>
-            <exclusions>
-                <exclusion>
-                    <artifactId>jnr-netdb</artifactId>
-                    <groupId>com.github.jnr</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>jnr-posix</artifactId>
-                    <groupId>com.github.jnr</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>jffi</artifactId>
-                    <groupId>com.github.jnr</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>nailgun-server</artifactId>
-                    <groupId>com.martiansoftware</groupId>
-                </exclusion>
-            </exclusions>
-        </dependency>        
-
-        <dependency>
-            <groupId>org.python</groupId>
-            <artifactId>jython-standalone</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-mock</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-processor-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-io</groupId>
-            <artifactId>commons-io</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-core-flowfile-attributes</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-stream-utils</artifactId>
-        </dependency>
-    </dependencies>
-</project>
-
-

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/processors/script/ExecuteScript.java
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/processors/script/ExecuteScript.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/processors/script/ExecuteScript.java
deleted file mode 100644
index 9058cf4..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/processors/script/ExecuteScript.java
+++ /dev/null
@@ -1,566 +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.script;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-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.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import javax.script.ScriptException;
-
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.components.Validator;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.io.BufferedInputStream;
-import org.apache.nifi.io.BufferedOutputStream;
-import org.apache.nifi.processor.AbstractProcessor;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.ProcessorInitializationContext;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.processor.annotation.CapabilityDescription;
-import org.apache.nifi.processor.annotation.EventDriven;
-import org.apache.nifi.processor.annotation.Tags;
-import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.io.InputStreamCallback;
-import org.apache.nifi.processor.io.StreamCallback;
-import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.scripting.ConverterScript;
-import org.apache.nifi.scripting.ReaderScript;
-import org.apache.nifi.scripting.Script;
-import org.apache.nifi.scripting.ScriptFactory;
-import org.apache.nifi.scripting.WriterScript;
-
-/**
- * <!-- Processor Documentation ================================================== -->
- * <h2>Description:</h2>
- * <p>
- * This processor provides the capability to execute scripts in various
- * scripting languages, and passes into the scripts the input stream and output
- * stream(s) representing an incoming flow file and any created flow files. The
- * processor is designed to be thread safe, so multiple concurrent tasks may
- * execute against a single script. The processor provides a framework which
- * enables script writers to implement 3 different types of scripts:
- * <ul>
- * ReaderScript - which enables stream-based reading of a FlowFile's
- * content</br> WriterScript - which enables stream-based reading and
- * writing/modifying of a FlowFile's content</br> ConverterScript - which
- * enables stream-based reading a FlowFile's content and stream-based writing to
- * newly created FlowFiles</br>
- * </ul>
- * Presently, the processor supports 3 scripting languages: Ruby, Python, and
- * JavaScript. The processor is built on the javax.script API which enables
- * ScriptEngine discovery, thread management, and encapsulates much of the low
- * level bridging-code that enables Java to Script language integration. Thus,
- * it is designed to be easily extended to other scripting languages. </br> The
- * attributes of a FlowFile and properties of the Processor are exposed to the
- * script by either a variable in the base class or a getter method. A script
- * may declare new Processor Properties and different Relationships via
- * overriding the getPropertyDescriptors and getRelationships methods,
- * respectively.
- * </p>
- * <p>
- * <strong>Properties:</strong>
- * </p>
- * <p>
- * In the list below, the names of required properties appear in bold. Any other
- * properties (not in bold) are considered optional. If a property has a default
- * value, it is indicated. If a property supports the use of the NiFi Expression
- * Language (or simply, "expression language"), that is also indicated. Of
- * particular note: This processor allows scripts to define additional Processor
- * properties, which will not be initially visible. Once the processor's
- * configuration is validated, script defined properties will become visible,
- * and may affect the validity of the processor.
- * </p>
- * <ul>
- * <li>
- * <strong>Script File Name</strong>
- * <ul>
- * <li>Script location, can be relative or absolute path.</li>
- * <li>Default value: no default</li>
- * <li>Supports expression language: false</li>
- * </ul>
- * </li>
- * <li>
- * <strong>Script Check Interval</strong>
- * <ul>
- * <li>The time period between checking for updates to a script.</li>
- * <li>Default value: 15 sec</li>
- * <li>Supports expression language: false</li>
- * </ul>
- * </li>
- * </ul>
- *
- * <p>
- * <strong>Relationships:</strong>
- * </p>
- * <p>
- * The initial 'out of the box' relationships are below. Of particular note is
- * the ability of a script to change the set of relationships. However, any
- * relationships defined by the script will not be visible until the processor's
- * configuration has been validated. Once done, new relationships will become
- * visible.
- * </p>
- * <ul>
- * <li>
- * success
- * <ul>
- * <li>Used when a file is successfully processed by a script.</li>
- * </ul>
- * </li>
- * <li>
- * failure
- * <ul>
- * <li>Used when an error occurs while processing a file with a script.</li>
- * </ul>
- * </li>
- * </ul>
- *
- * <p>
- * <strong>Example Scripts:</strong>
- * </p>
- * <ul>
- * JavaScript example - the 'with' statement imports packages defined in the
- * framework. Since the 'instance' variable is intended to be local scope (not
- * global), it must be named 'instance' as it it not passed back to the
- * processor upon script evaluation and must be fetched. If you make it global,
- * you can name it whatever you'd like...but this is intended to be
- * multi-threaded so do so at your own risk. Presently, there are issues with
- * the JavaScript scripting engine that prevent sub-classing the base classes in
- * the Processor's Java framework. So, what is actually happening is an instance
- * of the ReaderScript is created with a provided callback object. When we are
- * able to move to a more competent scripting engine, the code below will remain
- * the same, but the 'instance' variable will actually be a sub-class of
- * ReaderScript.
- *
- * <pre>
- *               with (Scripting) {
- *                 var instance = new ReaderScript({
- *                     route : function(input) {
- *                         var str = IOUtils.toString(input);
- *                         var expr = instance.getProperty("expr");
- *                         filename = instance.attributes.get("filename");
- *                         instance.setAttribute("filename", filename + ".modified");
- *                         if (str.match(expr)) {
- *                             return Script.FAIL_RELATIONSHIP;
- *                         } else {
- *                             return Script.SUCCESS_RELATIONSHIP;
- *                         }
- *                     }
- *                 });
- *               }
- * </pre>
- *
- * Ruby example - the 'OutputStreamHandler' is an interface which is called when
- * creating flow files.
- *
- * <pre>
- *                 java_import 'org.apache.nifi.scripting.OutputStreamHandler'
- *                 class SimpleConverter < ConverterScript
- *                   field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP, :logger, :attributes
- *
- *                   def convert(input)
- *                     in_io = input.to_io
- *                     createFlowFile("firstLine", FAIL_RELATIONSHIP, OutputStreamHandler.impl do |method, out|
- *                         out_io = out.to_io
- *                         out_io << in_io.readline.to_java_bytes
- *                         out_io.close
- *                         logger.debug("Wrote data to failure...this message logged with logger from super class")
- *                       end)
- *
- *                     createFlowFile("otherLines", SUCCESS_RELATIONSHIP, OutputStreamHandler.impl do |method, out|
- *                         out_io = out.to_io
- *                         in_io.each_line { |line|
- *                           out_io << line
- *                         }
- *                         out_io.close
- *                         logger.debug("Wrote data to success...this message logged with logger from super class")
- *                       end)
- *                     in_io.close
- *                   end
- *
- *                 end
- *
- *                 $logger.debug("Creating SimpleConverter...this message logged with logger from shared variables")
- *                 SimpleConverter.new
- * </pre>
- *
- * Python example - The difficulty with Python is that it does not return
- * objects upon script evaluation, so the instance of the Script class must be
- * fetched by name. Thus, you must define a variable called 'instance'.
- *
- * <pre>
- *                 import re
- *
- *                 class RoutingReader(ReaderScript):
- *                     A = Relationship.Builder().name("a").description("some good stuff").build()
- *                     B = Relationship.Builder().name("b").description("some other stuff").build()
- *                     C = Relationship.Builder().name("c").description("some bad stuff").build()
- *
- *                     def getRelationships(self):
- *                         return [self.A,self.B,self.C]
- *
- *                     def getExceptionRoute(self):
- *                         return self.C
- *
- *                     def route( self, input ):
- *                         for line in FileUtil.wrap(input):
- *                             if re.match("^bad", line, re.IGNORECASE):
- *                                 return self.B
- *                             if re.match("^sed", line):
- *                                 raise RuntimeError("That's no good!")
- *
- *                         return self.A
- *
- *                 instance = RoutingReader()
- * </pre>
- *
- * </ul>
- * <p>
- * <strong>Shared Variables</strong>
- * </p>
- * <ul>
- * <li>logger : global scope</li>
- * <li>properties : local/instance scope</li>
- * </ul>
- * <p>
- * <strong>Script API:</strong>
- * </p>
- * <ul>
- * <li>getAttribute(String) : String</li>
- * <li>getAttributes() : Map(String,String)</li>
- * <li>getExceptionRoute() : Relationship</li>
- * <li>getFileName() : String</li>
- * <li>getFlowFileEntryDate() : Calendar</li>
- * <li>getFlowFileSize() : long</li>
- * <li>getProperties() : Map(String, String)</li>
- * <li>getProperty(String) : String</li>
- * <li>getPropertyDescriptors() : List(PropertyDescriptor)</li>
- * <li>getRelationships() : Collection(Relationship)</li>
- * <li>getRoute() : Relationship</li>
- * <li>setRoute(Relationship)</li>
- * <li>setAttribute(String, String)</li>
- * <li>validate() : Collection(String)</li>
- * </ul>
- * <p>
- * <strong>ReaderScript API:</strong>
- * </p>
- * <ul>
- * <li>route(InputStream) : Relationship</li>
- * </ul>
- * <p>
- * <strong>WriterScript API:</strong>
- * </p>
- * <ul>
- * <li>process(InputStream, OutputStream)</li>
- * </ul>
- * <p>
- * <strong>ConverterScript API:</strong>
- * </p>
- * <ul>
- * <li>convert(InputStream)</li>
- * <li>createFlowFile(String, Relationship, OutputStreamHandler)</li>
- * </ul>
- * <p>
- * <strong>OutputStreamHandler API:</strong>
- * </p>
- * <ul>
- * <li>write(OutputStream)</li>
- * </ul>
- */
-@EventDriven
-@Tags({"script", "ruby", "python", "javascript", "execute"})
-@CapabilityDescription("Execute scripts in various scripting languages, and passes into the scripts the input stream and output stream(s) "
-        + "representing an incoming flow file and any created flow files.")
-public class ExecuteScript extends AbstractProcessor {
-
-    private final AtomicBoolean doCustomValidate = new AtomicBoolean(true);
-    private final AtomicReference<Set<Relationship>> relationships = new AtomicReference<>();
-    private final AtomicReference<List<PropertyDescriptor>> propertyDescriptors = new AtomicReference<>();
-    private volatile ScriptFactory scriptFactory;
-    private volatile Relationship exceptionRoute;
-
-    /**
-     * Script location, can be relative or absolute path -- passed as-is to
-     * {@link File#File(String) File constructor}
-     */
-    public static final PropertyDescriptor SCRIPT_FILE_NAME = new PropertyDescriptor.Builder()
-            .name("Script File Name")
-            .description("Script location, can be relative or absolute path")
-            .required(true)
-            .addValidator(new Validator() {
-
-                @Override
-                public ValidationResult validate(String subject, String input, ValidationContext context) {
-                    ValidationResult result = StandardValidators.FILE_EXISTS_VALIDATOR.validate(subject, input, context);
-                    if (result.isValid()) {
-                        int dotPos = input.lastIndexOf('.');
-                        if (dotPos < 1) {
-                            result = new ValidationResult.Builder()
-                            .subject(subject)
-                            .valid(false)
-                            .explanation("Filename must have an extension")
-                            .input(input)
-                            .build();
-                        }
-                    }
-                    return result;
-                }
-            })
-            .build();
-
-    static final PropertyDescriptor SCRIPT_CHECK_INTERVAL = new PropertyDescriptor.Builder()
-            .name("Script Check Interval")
-            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .description("The time period between checking for updates to a script")
-            .required(true)
-            .defaultValue("15 sec")
-            .build();
-
-    @Override
-    protected void init(ProcessorInitializationContext context) {
-        Set<Relationship> empty = Collections.emptySet();
-        relationships.set(empty);
-        ArrayList<PropertyDescriptor> propDescs = new ArrayList<>();
-        propDescs.add(SCRIPT_FILE_NAME);
-        propDescs.add(SCRIPT_CHECK_INTERVAL);
-        propertyDescriptors.set(Collections.unmodifiableList(propDescs));
-        scriptFactory = new ScriptFactory(getLogger());
-    }
-
-    @Override
-    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return propertyDescriptors.get();
-    }
-
-    @Override
-    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(String propertyDescriptorName) {
-        return new PropertyDescriptor.Builder()
-                .name(propertyDescriptorName)
-                .dynamic(true)
-                .addValidator(Validator.VALID)
-                .build();
-    }
-
-    @Override
-    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
-        doCustomValidate.set(true);
-    }
-
-    @Override
-    public Set<Relationship> getRelationships() {
-        return relationships.get();
-    }
-
-    /**
-     * Called by framework.
-     *
-     * Returns a list of reasons why this processor cannot be run.
-     * @return 
-     */
-    @Override
-    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
-        if (doCustomValidate.getAndSet(false)) {
-            long interval = validationContext.getProperty(SCRIPT_CHECK_INTERVAL).asTimePeriod(TimeUnit.MILLISECONDS);
-            scriptFactory.setScriptCheckIntervalMS(interval);
-            List<ValidationResult> results = new ArrayList<>();
-            String file = validationContext.getProperty(SCRIPT_FILE_NAME).getValue();
-            try {
-                Script s = scriptFactory.getScript(file);
-
-                // set the relationships of the processor
-                relationships.set(new HashSet<>(s.getRelationships()));
-
-                // need to get script's prop. descs. and validate. May, or may not, have dynamic
-                // props already...depends if this is the first time the processor is being configured.
-                Map<PropertyDescriptor, String> properties = validationContext.getProperties();
-
-                // need to compare props, if any, against script-expected props that are required.
-                // script may be expecting required props that are not known, or some props may have invalid
-                // values.
-                // processor may be configured with dynamic props that the script will use...but does not declare which would
-                // be a bad thing
-                List<PropertyDescriptor> scriptPropDescs = s.getPropertyDescriptors();
-                getLogger().debug("Script is {}", new Object[]{s});
-                getLogger().debug("Script file name is {}", new Object[]{s.getFileName()});
-                getLogger().debug("Script Prop Descs are: {}", new Object[]{scriptPropDescs.toString()});
-                getLogger().debug("Thread is: {}", new Object[]{Thread.currentThread().toString()});
-                for (PropertyDescriptor propDesc : scriptPropDescs) {
-                    // need to check for missing props
-                    if (propDesc.isRequired() && !properties.containsKey(propDesc)) {
-                        results.add(new ValidationResult.Builder()
-                                .subject("Script Properties")
-                                .valid(false)
-                                .explanation("Missing Property " + propDesc.getName())
-                                .build());
-
-                        // need to validate current value against script provided validator
-                    } else if (properties.containsKey(propDesc)) {
-                        String value = properties.get(propDesc);
-                        ValidationResult result = propDesc.validate(value, validationContext);
-                        if (!result.isValid()) {
-                            results.add(result);
-                        }
-                    } // else it is an optional prop according to the script and it is not specified by
-                    // the configuration of the processor
-                }
-
-                // need to update the known prop desc's with what we just got from the script
-                List<PropertyDescriptor> pds = new ArrayList<>(propertyDescriptors.get());
-                pds.addAll(scriptPropDescs);
-                propertyDescriptors.set(Collections.unmodifiableList(pds));
-
-                if (results.isEmpty()) {
-                    // so needed props are supplied and individually validated, now validate script
-                    Collection<String> reasons;
-                    reasons = s.validate();
-                    if (null == reasons) {
-                        getLogger().warn("Script had invalid return value for validate(), ignoring.");
-                    } else {
-                        for (String reason : reasons) {
-                            ValidationResult result = new ValidationResult.Builder()
-                                    .subject("ScriptValidation")
-                                    .valid(false)
-                                    .explanation(reason)
-                                    .build();
-                            results.add(result);
-                        }
-                    }
-                }
-
-                // get the exception route
-                exceptionRoute = s.getExceptionRoute();
-
-                return results;
-            } catch (ScriptException | IOException | NoSuchMethodException e) {
-                doCustomValidate.set(true);
-                results.add(new ValidationResult.Builder()
-                        .subject("ScriptValidation")
-                        .valid(false)
-                        .explanation("Cannot create script due to " + e.getMessage())
-                        .input(file)
-                        .build());
-                getLogger().error("Cannot create script due to " + e, e);
-                return results;
-            }
-        }
-        return null;
-    }
-
-    @Override
-    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
-        FlowFile flowFile = session.get();
-        if (flowFile == null) {
-            return; // fail-fast if there is no work to do
-        }
-
-        final String scriptFileName = context.getProperty(SCRIPT_FILE_NAME).getValue();
-        // doing this cloning because getProperties does not initialize props that have only their default values
-        // must do a getProperty for that value to be initialized
-        Map<String, String> props = new HashMap<>();
-        for (PropertyDescriptor propDesc : context.getProperties().keySet()) {
-            if (propDesc.isExpressionLanguageSupported()) {
-                props.put(propDesc.getName(), context.getProperty(propDesc).evaluateAttributeExpressions(flowFile).getValue());
-            } else {
-                props.put(propDesc.getName(), context.getProperty(propDesc).getValue());
-            }
-        }
-        Script script = null;
-        try {
-            final Script finalScript = scriptFactory.getScript(scriptFileName, props, flowFile);
-            script = finalScript;
-            if (finalScript instanceof ReaderScript) {
-                session.read(flowFile, new InputStreamCallback() {
-
-                    @Override
-                    public void process(InputStream in) throws IOException {
-                        try {
-                            ((ReaderScript) finalScript).process(new BufferedInputStream(in));
-                        } catch (NoSuchMethodException | ScriptException e) {
-                            getLogger().error("Failed to execute ReaderScript", e);
-                            throw new IOException(e);
-                        }
-                    }
-                });
-            } else if (finalScript instanceof WriterScript) {
-                flowFile = session.write(flowFile, new StreamCallback() {
-
-                    @Override
-                    public void process(InputStream in, OutputStream out) throws IOException {
-                        try {
-                            ((WriterScript) finalScript).process(new BufferedInputStream(in), new BufferedOutputStream(out));
-                            out.flush();
-                        } catch (NoSuchMethodException | ScriptException e) {
-                            getLogger().error("Failed to execute WriterScript", e);
-                            throw new IOException(e);
-                        }
-                    }
-                });
-            } else if (finalScript instanceof ConverterScript) {
-                ((ConverterScript) finalScript).process(session);
-
-                // Note that these scripts don't pass the incoming FF through,
-                // they always create new outputs
-                session.remove(flowFile);
-                return;
-            } else {
-                // only thing we can do is assume script has already run and done it's thing, so just transfer the incoming
-                // flowfile
-                getLogger().debug("Successfully executed script from {}", new Object[]{scriptFileName});
-            }
-
-            // update flow file attributes
-            flowFile = session.putAllAttributes(flowFile, finalScript.getAttributes());
-            Relationship route = finalScript.getRoute();
-            if (null == route) {
-                session.remove(flowFile);
-                getLogger().info("Removing flowfile {}", new Object[]{flowFile});
-            } else {
-                session.transfer(flowFile, route);
-                getLogger().info("Transferring flowfile {} to {}", new Object[]{flowFile, route});
-            }
-        } catch (ScriptException | IOException e) {
-            getLogger().error("Failed to create script from {} with flowFile {}. Rolling back session.",
-                    new Object[]{scriptFileName, flowFile}, e);
-            throw new ProcessException(e);
-        } catch (Exception e) {
-            if (null != script) {
-                getLogger().error("Failed to execute script from {}. Transferring flow file {} to {}",
-                        new Object[]{scriptFileName, flowFile, exceptionRoute}, e);
-                session.transfer(flowFile, exceptionRoute);
-            } else {
-                getLogger().error("Failed to execute script from {} with flowFile {}. Rolling back session",
-                        new Object[]{scriptFileName, flowFile}, e);
-                throw new ProcessException(e);
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ConverterScript.java
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ConverterScript.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ConverterScript.java
deleted file mode 100644
index 7be47a8..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ConverterScript.java
+++ /dev/null
@@ -1,131 +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.scripting;
-
-import java.io.ByteArrayOutputStream;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Map;
-
-import javax.script.Invocable;
-import javax.script.ScriptException;
-
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.io.BufferedInputStream;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.processor.io.InputStreamCallback;
-import org.apache.nifi.processor.io.OutputStreamCallback;
-
-/**
- * <p>
- * Script authors should extend this class if they want to perform complex
- * conversions in a NiFi processor.
- * </p>
- *
- * <p>
- * Scripts must implement {@link #convert(FileInputStream)}. This method may
- * create new FlowFiles and pass them to one or more routes. The input FlowFile
- * will be removed from the repository after execution of this method completes.
- * </p>
- *
- * <p>
- * In general, the {@link #convert(FileInputStream)} will read from the supplied
- * stream, then create one or more output sinks and route the result to the
- * relationship of choice using
- * {@link #routeStream(ByteArrayOutputStream, String, String)} or
- * {@link #routeBytes(byte[], String, String)}.
- *
- * <p>
- * Implement {@link #getProcessorRelationships()} to allow writing to
- * relationships other than <code>success</code> and <code>failure</code>. The
- * {@link #getRoute()} superclass method is *not* used by Converter Scripts.
- * </p>
- *
- */
-public class ConverterScript extends Script {
-
-    private ProcessSession session; // used to create files
-    private Object convertCallback;
-
-    public ConverterScript() {
-
-    }
-
-    public ConverterScript(Object... callbacks) {
-        super(callbacks);
-        for (Object callback : callbacks) {
-            if (callback instanceof Map<?, ?>) {
-                convertCallback = convertCallback == null && ((Map<?, ?>) callback).containsKey("convert") ? callback : convertCallback;
-            }
-        }
-    }
-
-    // Subclasses should implement this to define basic logic
-    protected void convert(InputStream stream) throws NoSuchMethodException, ScriptException {
-        if (convertCallback != null) {
-            ((Invocable) engine).invokeMethod(convertCallback, "convert", stream);
-        }
-    }
-
-    /**
-     * Owning processor uses this method to kick off handling of a single file
-     *
-     * @param aSession the owning processor's Repository (needed to make new
-     * files)
-     */
-    public void process(ProcessSession aSession) {
-        this.session = aSession;
-        this.session.read(this.flowFile, new InputStreamCallback() {
-
-            @Override
-            public void process(InputStream in) throws IOException {
-                BufferedInputStream stream = new BufferedInputStream(in);
-                try {
-                    convert(stream);
-                } catch (NoSuchMethodException | ScriptException e) {
-                    logger.error("Failed to execute 'convert' function in script", e);
-                    throw new IOException(e);
-                }
-            }
-        });
-    }
-
-    // this should go back to protected once we get Nashorn
-    public void createFlowFile(final String flowFileName, final Relationship relationship, final OutputStreamHandler handler) {
-        FlowFile result = session.create(this.flowFile);
-        result = session.putAttribute(result, CoreAttributes.FILENAME.key(), flowFileName);
-        try {
-            result = session.write(result, new OutputStreamCallback() {
-
-                @Override
-                public void process(OutputStream out) throws IOException {
-                    handler.write(out);
-                }
-            });
-            this.logger.info("Transfer flow file {} to {}", new Object[]{result, relationship});
-            session.transfer(result, relationship);
-        } catch (Exception e) {
-            this.logger.error("Could not create new flow file from script", e);
-            session.remove(result);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JRubyScriptFactory.java
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JRubyScriptFactory.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JRubyScriptFactory.java
deleted file mode 100644
index 883b688..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JRubyScriptFactory.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.scripting;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.apache.commons.io.FileUtils;
-
-public enum JRubyScriptFactory {
-
-    INSTANCE;
-
-    private static final String PRELOADS = "include Java\n"
-            + "\n"
-            + "java_import 'org.apache.nifi.components.PropertyDescriptor'\n"
-            + "java_import 'org.apache.nifi.components.Validator'\n"
-            + "java_import 'org.apache.nifi.processor.util.StandardValidators'\n"
-            + "java_import 'org.apache.nifi.processor.Relationship'\n"
-            + "java_import 'org.apache.nifi.logging.ProcessorLog'\n"
-            + "java_import 'org.apache.nifi.scripting.ReaderScript'\n"
-            + "java_import 'org.apache.nifi.scripting.WriterScript'\n"
-            + "java_import 'org.apache.nifi.scripting.ConverterScript'\n"
-            + "\n";
-
-    public String getScript(File scriptFile) throws IOException {
-        StringBuilder sb = new StringBuilder();
-        sb.append(PRELOADS)
-                .append(FileUtils.readFileToString(scriptFile, "UTF-8"));
-        return sb.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JavaScriptScriptFactory.java
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JavaScriptScriptFactory.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JavaScriptScriptFactory.java
deleted file mode 100644
index 774fb1f..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JavaScriptScriptFactory.java
+++ /dev/null
@@ -1,56 +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.scripting;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.StringUtils;
-
-public enum JavaScriptScriptFactory {
-
-    INSTANCE;
-
-    private static final String PRELOADS = "var Scripting = JavaImporter(\n"
-            + "        Packages.org.apache.nifi.components,\n"
-            + "        Packages.org.apache.nifi.processor.util,\n"
-            + "        Packages.org.apache.nifi.processor,\n"
-            + "        Packages.org.apache.nifi.logging,\n"
-            + "        Packages.org.apache.nifi.scripting,\n"
-            + "        Packages.org.apache.commons.io);\n"
-            + "var readFile = function (file) {\n"
-            + "  var script = Packages.org.apache.commons.io.FileUtils.readFileToString("
-            + "      new java.io.File($PATH, file)"
-            + "    );\n"
-            + "  return \"\" + script;\n"
-            + "}\n"
-            + "var require = function (file){\n"
-            + "  var exports={}, module={};\n"
-            + "  module.__defineGetter__('id', function(){return file;});"
-            + "  eval(readFile(file));\n"
-            + "  return exports;\n"
-            + "}\n";
-
-    public String getScript(File scriptFile) throws IOException {
-        StringBuilder sb = new StringBuilder();
-        final String parent = StringUtils.replace(scriptFile.getParent(), "\\", "/");
-        sb.append(PRELOADS).append("var $PATH = \"").append(parent).append("\"\n")
-                .append(FileUtils.readFileToString(scriptFile, "UTF-8"));
-        return sb.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JythonScriptFactory.java
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JythonScriptFactory.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JythonScriptFactory.java
deleted file mode 100644
index 6b40b5e..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JythonScriptFactory.java
+++ /dev/null
@@ -1,45 +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.scripting;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.apache.commons.io.FileUtils;
-
-public enum JythonScriptFactory {
-
-    INSTANCE;
-
-    private final static String PRELOADS = "from org.python.core.util import FileUtil\n"
-            + "from org.apache.nifi.components import PropertyDescriptor\n"
-            + "from org.apache.nifi.components import Validator\n"
-            + "from org.apache.nifi.processor.util import StandardValidators\n"
-            + "from org.apache.nifi.processor import Relationship\n"
-            + "from org.apache.nifi.logging import ProcessorLog\n"
-            + "from org.apache.nifi.scripting import ReaderScript\n"
-            + "from org.apache.nifi.scripting import WriterScript\n"
-            + "from org.apache.nifi.scripting import ConverterScript\n";
-
-    public String getScript(File scriptFile) throws IOException {
-        StringBuilder sb = new StringBuilder();
-        sb.append(PRELOADS)
-                .append(FileUtils.readFileToString(scriptFile, "UTF-8"));
-
-        return sb.toString();
-    }
-}


[08/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/RepositoryStatusReport.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/RepositoryStatusReport.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/RepositoryStatusReport.java
deleted file mode 100644
index e434905..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/RepositoryStatusReport.java
+++ /dev/null
@@ -1,28 +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;
-
-import java.util.Map;
-
-public interface RepositoryStatusReport {
-
-    void addReportEntry(FlowFileEvent entry);
-
-    Map<String, FlowFileEvent> getReportEntries();
-
-    FlowFileEvent getReportEntry(String componentId);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java
deleted file mode 100644
index 6f9c237..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.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.controller.service;
-
-import org.apache.nifi.controller.Availability;
-import org.apache.nifi.controller.ConfiguredComponent;
-import org.apache.nifi.controller.ControllerService;
-
-public interface ControllerServiceNode extends ConfiguredComponent {
-
-    ControllerService getControllerService();
-
-    Availability getAvailability();
-
-    void setAvailability(Availability availability);
-
-    boolean isDisabled();
-
-    void setDisabled(boolean disabled);
-
-    ControllerServiceReference getReferences();
-
-    void addReference(ConfiguredComponent referringComponent);
-
-    void removeReference(ConfiguredComponent referringComponent);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
deleted file mode 100644
index 35a255d..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.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.controller.service;
-
-import java.util.Map;
-
-import org.apache.nifi.controller.ControllerServiceLookup;
-
-/**
- *
- */
-public interface ControllerServiceProvider extends ControllerServiceLookup {
-
-    /**
-     * Gets the controller service for the specified identifier. Returns null if
-     * the identifier does not match a known service.
-     *
-     * @param type
-     * @param id
-     * @param properties
-     * @return
-     */
-    ControllerServiceNode createControllerService(String type, String id, Map<String, String> properties);
-
-    /**
-     * Gets the controller service node for the specified identifier. Returns
-     * <code>null</code> if the identifier does not match a known service
-     *
-     * @param id
-     * @return
-     */
-    ControllerServiceNode getControllerServiceNode(String id);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java
deleted file mode 100644
index 5cb676f..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java
+++ /dev/null
@@ -1,50 +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.service;
-
-import java.util.Set;
-
-import org.apache.nifi.controller.ConfiguredComponent;
-
-/**
- * Provides a collection of components that are referencing a Controller Service
- */
-public interface ControllerServiceReference {
-
-    /**
-     * Returns the component that is being referenced
-     *
-     * @return
-     */
-    ControllerServiceNode getReferencedComponent();
-
-    /**
-     * Returns a {@link Set} of all components that are referencing this
-     * Controller Service
-     *
-     * @return
-     */
-    Set<ConfiguredComponent> getReferencingComponents();
-
-    /**
-     * Returns a {@link Set} of all Processors and Reporting Tasks that are
-     * referencing the Controller Service and are running, in addition to all
-     *
-     * @return
-     */
-    Set<ConfiguredComponent> getRunningReferences();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java
deleted file mode 100644
index d1d5e5b..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java
+++ /dev/null
@@ -1,52 +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.events;
-
-import java.util.concurrent.atomic.AtomicLong;
-import org.apache.nifi.connectable.Connectable;
-import org.apache.nifi.reporting.Bulletin;
-
-/**
- *
- */
-public final class BulletinFactory {
-
-    private static final AtomicLong currentId = new AtomicLong(0);
-
-    public static Bulletin createBulletin(final Connectable connectable, final String category, final String severity, final String message) {
-        return BulletinFactory.createBulletin(connectable.getProcessGroup().getIdentifier(), connectable.getIdentifier(), connectable.getName(), category, severity, message);
-    }
-
-    public static Bulletin createBulletin(final String groupId, final String sourceId, final String sourceName, final String category, final String severity, final String message) {
-        final Bulletin bulletin = new ComponentBulletin(currentId.getAndIncrement());
-        bulletin.setGroupId(groupId);
-        bulletin.setSourceId(sourceId);
-        bulletin.setSourceName(sourceName);
-        bulletin.setCategory(category);
-        bulletin.setLevel(severity);
-        bulletin.setMessage(message);
-        return bulletin;
-    }
-
-    public static Bulletin createBulletin(final String category, final String severity, final String message) {
-        final Bulletin bulletin = new SystemBulletin(currentId.getAndIncrement());
-        bulletin.setCategory(category);
-        bulletin.setLevel(severity);
-        bulletin.setMessage(message);
-        return bulletin;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/BulletinProcessingStrategy.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/BulletinProcessingStrategy.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/BulletinProcessingStrategy.java
deleted file mode 100644
index 9846cf2..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/BulletinProcessingStrategy.java
+++ /dev/null
@@ -1,27 +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.events;
-
-import org.apache.nifi.reporting.Bulletin;
-
-/**
- *
- */
-public interface BulletinProcessingStrategy {
-
-    void update(Bulletin bulletin);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/ComponentBulletin.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/ComponentBulletin.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/ComponentBulletin.java
deleted file mode 100644
index 23c4cdb..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/ComponentBulletin.java
+++ /dev/null
@@ -1,30 +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.events;
-
-import org.apache.nifi.reporting.Bulletin;
-
-/**
- *
- */
-public class ComponentBulletin extends Bulletin {
-
-    ComponentBulletin(final long id) {
-        super(id);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/SystemBulletin.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/SystemBulletin.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/SystemBulletin.java
deleted file mode 100644
index f97dc46..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/SystemBulletin.java
+++ /dev/null
@@ -1,30 +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.events;
-
-import org.apache.nifi.reporting.Bulletin;
-
-/**
- *
- */
-public class SystemBulletin extends Bulletin {
-
-    SystemBulletin(final long id) {
-        super(id);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
deleted file mode 100644
index 61be59c..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
+++ /dev/null
@@ -1,723 +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.groups;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.nifi.connectable.Connectable;
-import org.apache.nifi.connectable.Connection;
-import org.apache.nifi.connectable.Funnel;
-import org.apache.nifi.connectable.Port;
-import org.apache.nifi.connectable.Position;
-import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.Snippet;
-import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.Processor;
-
-/**
- * <p>
- * ProcessGroup objects are containers for processing entities, such as
- * {@link Processor}s, {@link Port}s, and other {@link ProcessGroup}s.
- * </p>
- *
- * <p>
- * MUST BE THREAD-SAFE</p>
- */
-public interface ProcessGroup {
-
-    /**
-     * @return a reference to this ProcessGroup's parent. This will be
-     * <tt>null</tt> if and only if this is the root group.
-     */
-    ProcessGroup getParent();
-
-    /**
-     * Updates the ProcessGroup to point to a new parent
-     *
-     * @param group
-     */
-    void setParent(ProcessGroup group);
-
-    /**
-     * @return the ID of the ProcessGroup
-     */
-    String getIdentifier();
-
-    /**
-     * @return the name of the ProcessGroup
-     */
-    String getName();
-
-    /**
-     * Updates the name of this ProcessGroup.
-     *
-     * @param name
-     */
-    void setName(String name);
-
-    /**
-     * Updates the position of where this ProcessGroup is located in the graph
-     */
-    void setPosition(Position position);
-
-    /**
-     * Returns the position of where this ProcessGroup is located in the graph
-     *
-     * @return
-     */
-    Position getPosition();
-
-    /**
-     * @return the user-set comments about this ProcessGroup, or
-     * <code>null</code> if no comments have been set
-     */
-    String getComments();
-
-    /**
-     * Updates the comments for this ProcessGroup
-     *
-     * @param comments
-     */
-    void setComments(String comments);
-
-    /**
-     * Returns the counts for this ProcessGroup
-     *
-     * @return
-     */
-    ProcessGroupCounts getCounts();
-
-    /**
-     * Starts all Processors, Local Ports, and Funnels that are directly within
-     * this group and any child ProcessGroups, except for those that are
-     * disabled.
-     */
-    void startProcessing();
-
-    /**
-     * Stops all Processors, Local Ports, and Funnels that are directly within
-     * this group and child ProcessGroups, except for those that are disabled.
-     */
-    void stopProcessing();
-
-    /**
-     * Starts the given Processor
-     *
-     * @param processor the processor to start
-     * @throws IllegalStateException if the processor is not valid, or is
-     * already running
-     */
-    void enableProcessor(ProcessorNode processor);
-
-    /**
-     * Starts the given Input Port
-     *
-     * @param port
-     */
-    void enableInputPort(Port port);
-
-    /**
-     * Starts the given Output Port
-     *
-     * @param port
-     */
-    void enableOutputPort(Port port);
-
-    /**
-     * Starts the given Funnel
-     *
-     * @param funnel
-     */
-    void enableFunnel(Funnel funnel);
-
-    /**
-     * Starts the given Processor
-     *
-     * @param processor the processor to start
-     * @throws IllegalStateException if the processor is not valid, or is
-     * already running
-     */
-    void startProcessor(ProcessorNode processor);
-
-    /**
-     * Starts the given Input Port
-     *
-     * @param port
-     */
-    void startInputPort(Port port);
-
-    /**
-     * Starts the given Output Port
-     *
-     * @param port
-     */
-    void startOutputPort(Port port);
-
-    /**
-     * Starts the given Funnel
-     *
-     * @param funnel
-     */
-    void startFunnel(Funnel funnel);
-
-    /**
-     * Stops the given Processor
-     *
-     * @param processor
-     */
-    void stopProcessor(ProcessorNode processor);
-
-    /**
-     * Stops the given Port
-     *
-     * @param processor
-     */
-    void stopInputPort(Port port);
-
-    /**
-     * Stops the given Port
-     *
-     * @param processor
-     */
-    void stopOutputPort(Port port);
-
-    /**
-     * Stops the given Funnel
-     *
-     * @param processor
-     */
-    void stopFunnel(Funnel funnel);
-
-    /**
-     * Starts the given Processor
-     *
-     * @param processor the processor to start
-     * @throws IllegalStateException if the processor is not valid, or is
-     * already running
-     */
-    void disableProcessor(ProcessorNode processor);
-
-    /**
-     * Starts the given Input Port
-     *
-     * @param port
-     */
-    void disableInputPort(Port port);
-
-    /**
-     * Starts the given Output Port
-     *
-     * @param port
-     */
-    void disableOutputPort(Port port);
-
-    /**
-     * Starts the given Funnel
-     *
-     * @param funnel
-     */
-    void disableFunnel(Funnel funnel);
-
-    /**
-     * Indicates that the Flow is being shutdown; allows cleanup of resources
-     * associated with processors, etc.
-     */
-    void shutdown();
-
-    /**
-     * Returns a boolean indicating whether or not this ProcessGroup is the root
-     * group
-     *
-     * @return
-     */
-    boolean isRootGroup();
-
-    /**
-     * Adds a {@link Port} to be used for transferring {@link FlowFile}s from
-     * external sources to {@link Processor}s and other {@link Port}s within
-     * this ProcessGroup.
-     *
-     * @param port
-     */
-    void addInputPort(Port port);
-
-    /**
-     * Removes a {@link Port} from this ProcessGroup's list of Input Ports.
-     *
-     * @param port the Port to remove
-     * @throws NullPointerException if <code>port</code> is null
-     * @throws IllegalStateException if port is not an Input Port for this
-     * ProcessGroup
-     */
-    void removeInputPort(Port port);
-
-    /**
-     * @return the {@link Set} of all {@link Port}s that are used by this
-     * ProcessGroup as Input Ports.
-     */
-    Set<Port> getInputPorts();
-
-    /**
-     * @param id the ID of the input port
-     * @return the input port with the given ID, or <code>null</code> if it does
-     * not exist.
-     */
-    Port getInputPort(String id);
-
-    /**
-     * Adds a {@link Port} to be used for transferring {@link FlowFile}s to
-     * external sources.
-     *
-     * @param port the Port to add
-     */
-    void addOutputPort(Port port);
-
-    /**
-     * Removes a {@link Port} from this ProcessGroup's list of Output Ports.
-     *
-     * @param port the Port to remove
-     * @throws NullPointerException if <code>port</code> is null
-     * @throws IllegalStateException if port is not an Input Port for this
-     * ProcessGroup
-     */
-    void removeOutputPort(Port port);
-
-    /**
-     * @param id the ID of the output port
-     * @return the output port with the given ID, or <code>null</code> if it
-     * does not exist.
-     */
-    Port getOutputPort(String id);
-
-    /**
-     * @return the {@link Set} of all {@link Port}s that are used by this
-     * ProcessGroup as Output Ports.
-     */
-    Set<Port> getOutputPorts();
-
-    /**
-     * Adds a reference to a ProgressGroup as a child of this.
-     *
-     * @return the newly created reference
-     */
-    void addProcessGroup(ProcessGroup group);
-
-    /**
-     * Returns the ProcessGroup whose parent is <code>this</code> and whose id
-     * is given
-     *
-     * @param id
-     * @return
-     */
-    ProcessGroup getProcessGroup(String id);
-
-    /**
-     * @return a {@link Set} of all Process Group References that are contained
-     * within this.
-     */
-    Set<ProcessGroup> getProcessGroups();
-
-    /**
-     * @param group the group to remove
-     * @throws NullPointerException if <code>group</code> is null
-     * @throws IllegalStateException if group is not member of this
-     * ProcessGroup, or the given ProcessGroup is not empty (i.e., it contains
-     * at least one Processor, ProcessGroup, Input Port, Output Port, or Label).
-     */
-    void removeProcessGroup(ProcessGroup group);
-
-    /**
-     * Adds the already constructed processor instance to this group
-     *
-     * @param processor the processor to add
-     */
-    void addProcessor(ProcessorNode processor);
-
-    /**
-     * Removes the given processor from this group, destroying the Processor.
-     * The Processor is removed from the ProcessorRegistry, and any method in
-     * the Processor that is annotated with the
-     * {@link nifi.processor.annotation.OnRemoved OnRemoved} annotation will be
-     * invoked. All outgoing connections will also be destroyed
-     *
-     * @param processor the Processor to remove
-     * @throws NullPointerException if <code>processor</code> is null
-     * @throws IllegalStateException if <code>processor</code> is not a member
-     * of this ProcessGroup, is currently running, or has any incoming
-     * connections.
-     */
-    void removeProcessor(ProcessorNode processor);
-
-    /**
-     * @return a {@link Collection} of all FlowFileProcessors that are contained
-     * within this.
-     */
-    Set<ProcessorNode> getProcessors();
-
-    /**
-     * Returns the FlowFileProcessor with the given ID.
-     *
-     * @param id the ID of the processor to retrieve
-     * @return the processor with the given ID
-     * @throws NullPointerException if <code>id</code> is null.
-     */
-    ProcessorNode getProcessor(String id);
-
-    /**
-     * Returns the <code>Connectable</code> with the given ID, or
-     * <code>null</code> if the <code>Connectable</code> is not a member of the
-     * group
-     *
-     * @param id the ID of the Connectable
-     * @return
-     */
-    Connectable getConnectable(String id);
-
-    /**
-     * Adds the given connection to this ProcessGroup. This method also notifies
-     * the Source and Destination of the Connection that the Connection has been
-     * established.
-     *
-     * @param connection
-     * @throws NullPointerException if the connection is null
-     * @throws IllegalStateException if the source or destination of the
-     * connection is not a member of this ProcessGroup or if a connection
-     * already exists in this ProcessGroup with the same ID
-     */
-    void addConnection(Connection connection);
-
-    /**
-     * Removes the connection from this ProcessGroup.
-     *
-     * @param connection
-     * @throws IllegalStateException if <code>connection</code> is not contained
-     * within this.
-     */
-    void removeConnection(Connection connection);
-
-    /**
-     * Inherits a Connection from another ProcessGroup; this does not perform
-     * any validation but simply notifies the ProcessGroup that it is now the
-     * owner of the given Connection. This is used in place of the
-     * {@link #addConnection(Connection)} method when moving Connections from
-     * one group to another because addConnection notifies both the Source and
-     * Destination of the Connection that the Connection has been established;
-     * this method does not notify either, as both the Source and Destination
-     * should already be aware of the Connection.
-     *
-     * @param connection
-     */
-    void inheritConnection(Connection connection);
-
-    /**
-     * @return the Connection with the given ID, or <code>null</code> if the
-     * connection does not exist.
-     */
-    Connection getConnection(String id);
-
-    /**
-     * Returns the {@link Set} of all {@link Connection}s contained within this.
-     *
-     * @return
-     */
-    Set<Connection> getConnections();
-
-    /**
-     * Returns a List of all Connections contains within this ProcessGroup and
-     * any child ProcessGroups.
-     *
-     * @return
-     */
-    List<Connection> findAllConnections();
-
-    /**
-     * Adds the given RemoteProcessGroup to this ProcessGroup
-     *
-     * @param remoteGroup
-     *
-     * @throws NullPointerException if the given argument is null
-     */
-    void addRemoteProcessGroup(RemoteProcessGroup remoteGroup);
-
-    /**
-     * Removes the given RemoteProcessGroup from this ProcessGroup
-     *
-     * @param remoteGroup
-     * @throws NullPointerException if the argument is null
-     * @throws IllegalStateException if the given argument does not belong to
-     * this ProcessGroup
-     */
-    void removeRemoteProcessGroup(RemoteProcessGroup remoteGroup);
-
-    /**
-     * Returns the RemoteProcessGroup that is the child of this ProcessGroup and
-     * has the given ID. If no RemoteProcessGroup can be found with the given
-     * ID, returns <code>null</code>.
-     *
-     * @param id
-     * @return
-     */
-    RemoteProcessGroup getRemoteProcessGroup(String id);
-
-    /**
-     * Returns a set of all RemoteProcessGroups that belong to this
-     * ProcessGroup. If no RemoteProcessGroup's have been added to this
-     * ProcessGroup, will return an empty Set.
-     *
-     * @return
-     */
-    Set<RemoteProcessGroup> getRemoteProcessGroups();
-
-    /**
-     * Adds the given Label to this ProcessGroup
-     *
-     * @param label the label to add
-     * @return
-     *
-     * @throws NullPointerException if the argument is null
-     */
-    void addLabel(Label label);
-
-    /**
-     * Removes the given Label from this ProcessGroup
-     *
-     * @param label the label to remove
-     * @throws NullPointerException if the argument is null
-     * @throws IllegalStateException if the given argument does not belong to
-     * this ProcessGroup
-     */
-    void removeLabel(Label label);
-
-    /**
-     * Returns a set of all Labels that belong to this ProcessGroup. If no
-     * Labels belong to this ProcessGroup, returns an empty Set.
-     *
-     * @return
-     */
-    Set<Label> getLabels();
-
-    /**
-     * Returns the Label that belongs to this ProcessGroup and has the given id.
-     * If no Label can be found with this ID, returns <code>null</code>.
-     *
-     * @param id
-     * @return
-     */
-    Label getLabel(String id);
-
-    /**
-     * Returns the Process Group with the given ID, if it exists as a child of
-     * this ProcessGroup, or is this ProcessGroup. This performs a recursive
-     * search of all ProcessGroups and descendant ProcessGroups
-     *
-     * @param id
-     * @return
-     */
-    ProcessGroup findProcessGroup(String id);
-
-    /**
-     * Returns the RemoteProcessGroup with the given ID, if it exists as a child
-     * or descendant of this ProcessGroup. This performs a recursive search of
-     * all ProcessGroups and descendant ProcessGroups
-     *
-     * @param id
-     * @return
-     */
-    RemoteProcessGroup findRemoteProcessGroup(String id);
-
-    /**
-     * Returns a List of all Remote Process Groups that are children or
-     * descendants of this ProcessGroup. This performs a recursive search of all
-     * descendant ProcessGroups
-     *
-     * @return
-     */
-    List<RemoteProcessGroup> findAllRemoteProcessGroups();
-
-    /**
-     * Returns the Processor with the given ID, if it exists as a child or
-     * descendant of this ProcessGroup. This performs a recursive search of all
-     * descendant ProcessGroups
-     *
-     * @param id
-     * @return
-     */
-    ProcessorNode findProcessor(String id);
-
-    /**
-     * Returns a List of all Processors that are children or descendants of this
-     * ProcessGroup. This performs a recursive search of all descendant
-     * ProcessGroups
-     *
-     * @return
-     */
-    List<ProcessorNode> findAllProcessors();
-
-    /**
-     * Returns a List of all Labels that are children or descendants of this
-     * ProcessGroup. This performsn a recursive search of all descendant
-     * ProcessGroups
-     *
-     * @return
-     */
-    List<Label> findAllLabels();
-
-    /**
-     * Returns the input port with the given ID, if it exists; otherwise returns
-     * null. This performs a recursive search of all Input Ports and descendant
-     * ProcessGroups
-     *
-     * @param id
-     * @return
-     */
-    Port findInputPort(String id);
-
-    /**
-     * Returns the input port with the given name, if it exists; otherwise
-     * returns null. ProcessGroups
-     *
-     * @param name
-     * @return
-     */
-    Port getInputPortByName(String name);
-
-    /**
-     * Returns the output port with the given ID, if it exists; otherwise
-     * returns null. This performs a recursive search of all Output Ports and
-     * descendant ProcessGroups
-     *
-     * @param id
-     * @return
-     */
-    Port findOutputPort(String id);
-
-    /**
-     * Returns the output port with the given name, if it exists; otherwise
-     * returns null.
-     *
-     * @param name
-     * @return
-     */
-    Port getOutputPortByName(String name);
-
-    /**
-     * Adds the given funnel to this ProcessGroup
-     *
-     * @param funnel
-     */
-    void addFunnel(Funnel funnel);
-
-    /**
-     * Returns a Set of all Funnels that belong to this ProcessGroup
-     *
-     * @return
-     */
-    Set<Funnel> getFunnels();
-
-    /**
-     * Returns the funnel with the given identifier
-     *
-     * @param id
-     * @return
-     */
-    Funnel getFunnel(String id);
-
-    /**
-     * Removes the given funnel from this ProcessGroup
-     *
-     * @param funnel
-     *
-     * @throws IllegalStateException if the funnel is not a member of this
-     * ProcessGroup or has incoming or outgoing connections
-     */
-    void removeFunnel(Funnel funnel);
-
-    /**
-     * @return <code>true</code> if this ProcessGroup has no Processors, Labels,
-     * Connections, ProcessGroups, RemoteProcessGroupReferences, or Ports.
-     * Otherwise, returns <code>false</code>.
-     */
-    boolean isEmpty();
-
-    /**
-     * Removes all of the components whose ID's are specified within the given
-     * {@link Snippet} from this ProcessGroup.
-     *
-     * @param snippet
-     *
-     * @throws NullPointerException if argument is null
-     * @throws IllegalStateException if any ID in the snippet refers to a
-     * component that is not within this ProcessGroup
-     */
-    void remove(final Snippet snippet);
-
-    /**
-     * Returns the Connectable with the given ID, if it exists; otherwise
-     * returns null. This performs a recursive search of all ProcessGroups'
-     * input ports, output ports, funnels, processors, and remote process groups
-     *
-     * @param identifier
-     * @return
-     */
-    Connectable findConnectable(String identifier);
-
-    /**
-     * Moves all of the components whose ID's are specified within the given
-     * {@link Snippet} from this ProcessGroup into the given destination
-     * ProcessGroup
-     *
-     * @param snippet
-     * @param destination
-     *
-     * @throws NullPointerExcepiton if either argument is null
-     * @throws IllegalStateException if any ID in the snippet refers to a
-     * component that is not within this ProcessGroup
-     */
-    void move(final Snippet snippet, final ProcessGroup destination);
-
-    void verifyCanDelete();
-
-    void verifyCanStart();
-
-    void verifyCanStop();
-
-    /**
-     * Ensures that deleting the given snippet is a valid operation at this
-     * point in time, depending on the state of this ProcessGroup
-     *
-     * @param snippet
-     *
-     * @throws IllegalStateException if deleting the Snippet is not valid at
-     * this time
-     */
-    void verifyCanDelete(Snippet snippet);
-
-    /**
-     * Ensure that moving the given snippet to the given new group is a valid
-     * operation at this point in time, depending on the state of both
-     * ProcessGroups
-     *
-     * @param snippet
-     * @param newProcessGroup
-     *
-     * @throws IllegalStateException if the move is not valid at this time
-     */
-    void verifyCanMove(Snippet snippet, ProcessGroup newProcessGroup);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/ProcessGroupCounts.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/ProcessGroupCounts.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/ProcessGroupCounts.java
deleted file mode 100644
index 3eb594b..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/ProcessGroupCounts.java
+++ /dev/null
@@ -1,66 +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.groups;
-
-public class ProcessGroupCounts {
-
-    private final int inputPortCount, outputPortCount, runningCount, stoppedCount, invalidCount, disabledCount, activeRemotePortCount, inactiveRemotePortCount;
-
-    public ProcessGroupCounts(final int inputPortCount, final int outputPortCount, final int runningCount,
-            final int stoppedCount, final int invalidCount, final int disabledCount, final int activeRemotePortCount, final int inactiveRemotePortCount) {
-        this.inputPortCount = inputPortCount;
-        this.outputPortCount = outputPortCount;
-        this.runningCount = runningCount;
-        this.stoppedCount = stoppedCount;
-        this.invalidCount = invalidCount;
-        this.disabledCount = disabledCount;
-        this.activeRemotePortCount = activeRemotePortCount;
-        this.inactiveRemotePortCount = inactiveRemotePortCount;
-    }
-
-    public int getInputPortCount() {
-        return inputPortCount;
-    }
-
-    public int getOutputPortCount() {
-        return outputPortCount;
-    }
-
-    public int getRunningCount() {
-        return runningCount;
-    }
-
-    public int getStoppedCount() {
-        return stoppedCount;
-    }
-
-    public int getInvalidCount() {
-        return invalidCount;
-    }
-
-    public int getDisabledCount() {
-        return disabledCount;
-    }
-
-    public int getActiveRemotePortCount() {
-        return activeRemotePortCount;
-    }
-
-    public int getInactiveRemotePortCount() {
-        return inactiveRemotePortCount;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
deleted file mode 100644
index e0cca64..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
+++ /dev/null
@@ -1,255 +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.groups;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.Date;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.connectable.Position;
-import org.apache.nifi.controller.exception.CommunicationsException;
-import org.apache.nifi.events.EventReporter;
-import org.apache.nifi.remote.PeerStatus;
-import org.apache.nifi.remote.RemoteGroupPort;
-import org.apache.nifi.remote.protocol.CommunicationsSession;
-
-public interface RemoteProcessGroup {
-
-    String getIdentifier();
-
-    URI getTargetUri();
-
-    ProcessGroup getProcessGroup();
-
-    void setProcessGroup(ProcessGroup group);
-
-    void setPosition(Position position);
-
-    Position getPosition();
-
-    String getComments();
-
-    void setComments(String comments);
-
-    void shutdown();
-    
-    /**
-     * Returns the name of this RemoteProcessGroup. The value returned will
-     * never be null. If unable to communicate with the remote instance, the URI
-     * of that instance may be returned instead
-     *
-     * @return
-     */
-    String getName();
-
-    void setName(String name);
-
-    void setInputPorts(Set<RemoteProcessGroupPortDescriptor> ports);
-
-    void setOutputPorts(Set<RemoteProcessGroupPortDescriptor> ports);
-
-    Set<RemoteGroupPort> getInputPorts();
-
-    Set<RemoteGroupPort> getOutputPorts();
-
-    RemoteGroupPort getInputPort(String id);
-
-    RemoteGroupPort getOutputPort(String id);
-
-    ProcessGroupCounts getCounts();
-
-    void refreshFlowContents() throws CommunicationsException;
-
-    Date getLastRefreshTime();
-
-    void setYieldDuration(final String yieldDuration);
-
-    String getYieldDuration();
-
-    /**
-     * Sets the timeout using the TimePeriod format (e.g., "30 secs", "1 min")
-     *
-     * @param timePeriod
-     * @throws IllegalArgumentException
-     */
-    void setCommunicationsTimeout(String timePeriod) throws IllegalArgumentException;
-
-    /**
-     * Returns the communications timeout in terms of the given TimeUnit
-     *
-     * @param timeUnit
-     * @return
-     */
-    int getCommunicationsTimeout(TimeUnit timeUnit);
-
-    /**
-     * Returns the user-configured String representation of the communications
-     * timeout
-     *
-     * @return
-     */
-    String getCommunicationsTimeout();
-
-    /**
-     * @return the port that the remote instance is listening on for
-     * site-to-site communication, or <code>null</code> if the remote instance
-     * is not configured to allow site-to-site communications.
-     *
-     * @throws IOException if unable to communicate with the remote instance
-     */
-    Integer getListeningPort() throws IOException;
-
-    /**
-     * Indicates whether or not the RemoteProcessGroup is currently scheduled to
-     * transmit data
-     *
-     * @return
-     */
-    boolean isTransmitting();
-
-    /**
-     * Initiates communications between this instance and the remote instance.
-     */
-    void startTransmitting();
-
-    /**
-     * Immediately terminates communications between this instance and the
-     * remote instance.
-     */
-    void stopTransmitting();
-
-    /**
-     * Initiates communications between this instance and the remote instance
-     * only for the port specified.
-     *
-     * @param port
-     */
-    void startTransmitting(RemoteGroupPort port);
-
-    /**
-     * Immediately terminates communications between this instance and the
-     * remote instance only for the port specified.
-     *
-     * @param port
-     */
-    void stopTransmitting(RemoteGroupPort port);
-
-    /**
-     * Indicates whether or not communications with this RemoteProcessGroup will
-     * be secure (2-way authentication)
-     *
-     * @return
-     */
-    boolean isSecure() throws CommunicationsException;
-
-    /**
-     * Indicates whether or not communications with this RemoteProcessGroup will
-     * be secure (2-way authentication). Returns null if unknown.
-     *
-     * @return
-     */
-    Boolean getSecureFlag();
-
-    /**
-     * Returns true if the target system has site to site enabled. Returns false
-     * otherwise (they don't or they have not yet responded).
-     *
-     * @return
-     */
-    boolean isSiteToSiteEnabled();
-
-    /**
-     * Returns a String indicating why we are not authorized to communicate with
-     * the remote instance, or <code>null</code> if we are authorized
-     *
-     * @return
-     */
-    String getAuthorizationIssue();
-
-    /**
-     * Returns the {@link EventReporter} that can be used to report any notable
-     * events
-     *
-     * @return
-     */
-    EventReporter getEventReporter();
-
-    /**
-     * Initiates a task in the remote process group to re-initialize, as a
-     * result of clustering changes
-     *
-     * @param isClustered whether or not this instance is now clustered
-     */
-    void reinitialize(boolean isClustered);
-
-    /**
-     * Removes all non existent ports from this RemoteProcessGroup.
-     */
-    void removeAllNonExistentPorts();
-
-    /**
-     * Removes a port that no longer exists on the remote instance from this
-     * RemoteProcessGroup
-     *
-     * @param port
-     */
-    void removeNonExistentPort(final RemoteGroupPort port);
-
-    /**
-     *
-     * @return @throws IOException
-     */
-    CommunicationsSession establishSiteToSiteConnection() throws IOException;
-
-    /**
-     * Called whenever RemoteProcessGroup is removed from the flow, so that any
-     * resources can be cleaned up appropriately.
-     */
-    void onRemove();
-
-    void verifyCanDelete();
-
-    void verifyCanDelete(boolean ignoreConnections);
-
-    void verifyCanStartTransmitting();
-
-    void verifyCanStopTransmitting();
-
-    void verifyCanUpdate();
-
-    /**
-     * Returns a set of PeerStatus objects that describe the different peers
-     * that we can communicate with for this RemoteProcessGroup.
-     *
-     * If the destination is a cluster, this set will contain PeerStatuses for
-     * each of the nodes in the cluster.
-     *
-     * If the destination is a standalone instance, this set will contain just a
-     * PeerStatus for the destination.
-     *
-     * Once the PeerStatuses have been obtained, they may be cached by this
-     * RemoteProcessGroup for some amount of time.
-     *
-     * If unable to obtain the PeerStatuses or no peer status has yet been
-     * obtained, will return null.
-     *
-     * @return
-     */
-    Set<PeerStatus> getPeerStatuses();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroupPortDescriptor.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroupPortDescriptor.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroupPortDescriptor.java
deleted file mode 100644
index fb4f6e0..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroupPortDescriptor.java
+++ /dev/null
@@ -1,92 +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.groups;
-
-public interface RemoteProcessGroupPortDescriptor {
-
-    /**
-     * The comments as configured in the target port.
-     *
-     * @return
-     */
-    String getComments();
-
-    /**
-     * The number tasks that may transmit flow files to the target port
-     * concurrently.
-     *
-     * @return
-     */
-    Integer getConcurrentlySchedulableTaskCount();
-
-    /**
-     * The id of the target port.
-     *
-     * @return
-     */
-    String getId();
-
-    /**
-     * The id of the remote process group that this port resides in.
-     *
-     * @return
-     */
-    String getGroupId();
-
-    /**
-     * The name of the target port.
-     *
-     * @return
-     */
-    String getName();
-
-    /**
-     * Whether or not this remote group port is configured for transmission.
-     *
-     * @return
-     */
-    Boolean isTransmitting();
-
-    /**
-     * Whether or not flow file are compressed when sent to this target port.
-     *
-     * @return
-     */
-    Boolean getUseCompression();
-
-    /**
-     * Whether ot not the target port exists.
-     *
-     * @return
-     */
-    Boolean getExists();
-
-    /**
-     * Whether or not the target port is running.
-     *
-     * @return
-     */
-    Boolean isTargetRunning();
-
-    /**
-     * Whether or not this port has either an incoming or outgoing connection.
-     *
-     * @return
-     */
-    Boolean isConnected();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogMessage.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogMessage.java
deleted file mode 100644
index 2ac89de..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogMessage.java
+++ /dev/null
@@ -1,74 +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.logging;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.sql.Date;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.Locale;
-
-public class LogMessage {
-
-    private final String message;
-    private final LogLevel level;
-    private final Throwable throwable;
-    private final long time;
-
-    public static final String DATE_TIME_FORMAT = "yyyy-MM-dd HH:mm:ss.SSS";
-    public static final String TO_STRING_FORMAT = "%1$s %2$s - %3$s";
-
-    public LogMessage(final long millisSinceEpoch, final LogLevel level, final String message, final Throwable throwable) {
-        this.level = level;
-        this.throwable = throwable;
-        this.message = message;
-        this.time = millisSinceEpoch;
-    }
-
-    public long getMillisSinceEpoch() {
-        return time;
-    }
-
-    public String getMessage() {
-        return message;
-    }
-
-    public LogLevel getLevel() {
-        return level;
-    }
-
-    public Throwable getThrowable() {
-        return throwable;
-    }
-
-    @Override
-    public String toString() {
-        final DateFormat dateFormat = new SimpleDateFormat(DATE_TIME_FORMAT, Locale.US);
-        final String formattedTime = dateFormat.format(new Date(time));
-
-        String formattedMsg = String.format(TO_STRING_FORMAT, formattedTime, level.toString(), message);
-        if (throwable != null) {
-            final StringWriter sw = new StringWriter();
-            final PrintWriter pw = new PrintWriter(sw);
-            throwable.printStackTrace(pw);
-            formattedMsg += "\n" + sw.toString();
-        }
-
-        return formattedMsg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogObserver.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogObserver.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogObserver.java
deleted file mode 100644
index a75f8ea..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogObserver.java
+++ /dev/null
@@ -1,22 +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.logging;
-
-public interface LogObserver {
-
-    void onLogMessage(LogMessage message);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogRepository.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogRepository.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogRepository.java
deleted file mode 100644
index 4a017ce..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogRepository.java
+++ /dev/null
@@ -1,67 +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.logging;
-
-public interface LogRepository {
-
-    void addLogMessage(LogLevel level, String message);
-
-    void addLogMessage(LogLevel level, String message, Throwable t);
-
-    void addLogMessage(LogLevel level, String messageFormat, Object[] params);
-
-    void addLogMessage(LogLevel level, String messageFormat, Object[] params, Throwable t);
-
-    /**
-     * Registers an observer so that it will be notified of all Log Messages
-     * whose levels are at least equal to the given level.
-     *
-     * @param observerIdentifier
-     * @param level
-     * @param observer
-     */
-    void addObserver(String observerIdentifier, LogLevel level, LogObserver observer);
-
-    /**
-     * Sets the observation level of the specified observer.
-     *
-     * @param observerIdentifier
-     * @param level
-     */
-    void setObservationLevel(String observerIdentifier, LogLevel level);
-
-    /**
-     * Gets the observation level for the specified observer.
-     *
-     * @param observerIdentifier
-     * @return
-     */
-    LogLevel getObservationLevel(String observerIdentifier);
-
-    /**
-     * Removes the given LogObserver from this Repository.
-     *
-     * @param observerIdentifier
-     * @return 
-     */
-    LogObserver removeObserver(String observerIdentifier);
-
-    /**
-     * Removes all LogObservers from this Repository
-     */
-    void removeAllObservers();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogRepositoryFactory.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogRepositoryFactory.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogRepositoryFactory.java
deleted file mode 100644
index 76ca661..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogRepositoryFactory.java
+++ /dev/null
@@ -1,61 +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.logging;
-
-import static java.util.Objects.requireNonNull;
-
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import org.slf4j.LoggerFactory;
-
-@SuppressWarnings("unchecked")
-public class LogRepositoryFactory {
-
-    public static final String LOG_REPOSITORY_CLASS_NAME = "org.apache.nifi.logging.repository.StandardLogRepository";
-
-    private static final ConcurrentMap<String, LogRepository> repositoryMap = new ConcurrentHashMap<>();
-    private static final Class<LogRepository> logRepositoryClass;
-
-    static {
-        Class<LogRepository> clazz = null;
-        try {
-            clazz = (Class<LogRepository>) Class.forName(LOG_REPOSITORY_CLASS_NAME, true, LogRepositoryFactory.class.getClassLoader());
-        } catch (ClassNotFoundException e) {
-            LoggerFactory.getLogger(LogRepositoryFactory.class).error("Unable to find class {}; logging may not work properly", LOG_REPOSITORY_CLASS_NAME);
-        }
-        logRepositoryClass = clazz;
-    }
-
-    public static LogRepository getRepository(final String processorId) {
-        LogRepository repository = repositoryMap.get(requireNonNull(processorId));
-        if (repository == null) {
-            try {
-                repository = logRepositoryClass.newInstance();
-            } catch (final Exception e) {
-                throw new RuntimeException(e);
-            }
-
-            final LogRepository oldRepository = repositoryMap.putIfAbsent(processorId, repository);
-            if (oldRepository != null) {
-                repository = oldRepository;
-            }
-        }
-
-        return repository;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/nar/NarCloseable.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/nar/NarCloseable.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/nar/NarCloseable.java
deleted file mode 100644
index b25c90b..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/nar/NarCloseable.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.nar;
-
-import java.io.Closeable;
-
-/**
- *
- */
-public class NarCloseable implements Closeable {
-
-    public static NarCloseable withNarLoader() {
-        final ClassLoader current = Thread.currentThread().getContextClassLoader();
-        Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());
-        return new NarCloseable(current);
-    }
-
-    private final ClassLoader toSet;
-
-    private NarCloseable(final ClassLoader toSet) {
-        this.toSet = toSet;
-    }
-
-    @Override
-    public void close() {
-        if (toSet != null) {
-            Thread.currentThread().setContextClassLoader(toSet);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
deleted file mode 100644
index aa905a8..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
+++ /dev/null
@@ -1,188 +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.nar;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.ArrayList;
-import java.util.Enumeration;
-import java.util.List;
-
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.components.Validator;
-import org.apache.nifi.controller.ControllerService;
-import org.apache.nifi.controller.repository.ContentRepository;
-import org.apache.nifi.controller.repository.FlowFileRepository;
-import org.apache.nifi.controller.repository.FlowFileSwapManager;
-import org.apache.nifi.controller.status.history.ComponentStatusRepository;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.io.InputStreamCallback;
-import org.apache.nifi.processor.io.OutputStreamCallback;
-import org.apache.nifi.processor.io.StreamCallback;
-import org.apache.nifi.provenance.ProvenanceEventRepository;
-import org.apache.nifi.reporting.ReportingTask;
-
-/**
- *
- * @author none THREAD SAFE
- */
-public class NarThreadContextClassLoader extends URLClassLoader {
-
-    static final ContextSecurityManager contextSecurityManager = new ContextSecurityManager();
-    private final ClassLoader forward = ClassLoader.getSystemClassLoader();
-    private static final List<Class<?>> narSpecificClasses = new ArrayList<>();
-
-    static {
-        narSpecificClasses.add(Processor.class);
-        narSpecificClasses.add(FlowFilePrioritizer.class);
-        narSpecificClasses.add(ReportingTask.class);
-        narSpecificClasses.add(Validator.class);
-        narSpecificClasses.add(InputStreamCallback.class);
-        narSpecificClasses.add(OutputStreamCallback.class);
-        narSpecificClasses.add(StreamCallback.class);
-        narSpecificClasses.add(ControllerService.class);
-        narSpecificClasses.add(AuthorityProvider.class);
-        narSpecificClasses.add(ProvenanceEventRepository.class);
-        narSpecificClasses.add(ComponentStatusRepository.class);
-        narSpecificClasses.add(FlowFileRepository.class);
-        narSpecificClasses.add(FlowFileSwapManager.class);
-        narSpecificClasses.add(ContentRepository.class);
-    }
-
-    private NarThreadContextClassLoader() {
-        super(new URL[0]);
-    }
-
-    @Override
-    public void clearAssertionStatus() {
-        lookupClassLoader().clearAssertionStatus();
-    }
-
-    @Override
-    public URL getResource(String name) {
-        return lookupClassLoader().getResource(name);
-    }
-
-    @Override
-    public InputStream getResourceAsStream(String name) {
-        return lookupClassLoader().getResourceAsStream(name);
-    }
-
-    @Override
-    public Enumeration<URL> getResources(String name) throws IOException {
-        return lookupClassLoader().getResources(name);
-    }
-
-    @Override
-    public Class<?> loadClass(String name) throws ClassNotFoundException {
-        return lookupClassLoader().loadClass(name);
-    }
-
-    @Override
-    public void setClassAssertionStatus(String className, boolean enabled) {
-        lookupClassLoader().setClassAssertionStatus(className, enabled);
-    }
-
-    @Override
-    public void setDefaultAssertionStatus(boolean enabled) {
-        lookupClassLoader().setDefaultAssertionStatus(enabled);
-    }
-
-    @Override
-    public void setPackageAssertionStatus(String packageName, boolean enabled) {
-        lookupClassLoader().setPackageAssertionStatus(packageName, enabled);
-    }
-
-    private ClassLoader lookupClassLoader() {
-        final Class<?>[] classStack = contextSecurityManager.getExecutionStack();
-
-        for (Class<?> currentClass : classStack) {
-            final Class<?> narClass = findNarClass(currentClass);
-            if (narClass != null) {
-                final ClassLoader desiredClassLoader = narClass.getClassLoader();
-
-                // When new Threads are created, the new Thread inherits the ClassLoaderContext of
-                // the caller. However, the call stack of that new Thread may not trace back to any NiFi-specific
-                // code. Therefore, the NarThreadContextClassLoader will be unable to find the appropriate NAR
-                // ClassLoader. As a result, we want to set the ContextClassLoader to the NAR ClassLoader that
-                // contains the class or resource that we are looking for.
-                // This locks the current Thread into the appropriate NAR ClassLoader Context. The framework will change
-                // the ContextClassLoader back to the NarThreadContextClassLoader as appropriate via the
-                // {@link FlowEngine.beforeExecute(Thread, Runnable)} and 
-                // {@link FlowEngine.afterExecute(Thread, Runnable)} methods.
-                if (desiredClassLoader instanceof NarClassLoader) {
-                    Thread.currentThread().setContextClassLoader(desiredClassLoader);
-                }
-                return desiredClassLoader;
-            }
-        }
-        return forward;
-    }
-
-    private Class<?> findNarClass(final Class<?> cls) {
-        for (final Class<?> narClass : narSpecificClasses) {
-            if (narClass.isAssignableFrom(cls)) {
-                return cls;
-            } else if (cls.getEnclosingClass() != null) {
-                return findNarClass(cls.getEnclosingClass());
-            }
-        }
-
-        return null;
-    }
-
-    private static class SingletonHolder {
-
-        public static final NarThreadContextClassLoader instance = new NarThreadContextClassLoader();
-    }
-
-    public static NarThreadContextClassLoader getInstance() {
-        return SingletonHolder.instance;
-    }
-
-    static class ContextSecurityManager extends SecurityManager {
-
-        Class<?>[] getExecutionStack() {
-            return getClassContext();
-        }
-    }
-
-    public static <T> T createInstance(final String implementationClassName, final Class<T> typeDefinition) throws InstantiationException, IllegalAccessException, ClassNotFoundException {
-        final ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
-        Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());
-        try {
-            final ClassLoader detectedClassLoaderForType = ExtensionManager.getClassLoader(implementationClassName);
-            final Class<?> rawClass;
-            if (detectedClassLoaderForType == null) {
-                // try to find from the current class loader
-                rawClass = Class.forName(implementationClassName);
-            } else {
-                // try to find from the registered classloader for that type
-                rawClass = Class.forName(implementationClassName, true, ExtensionManager.getClassLoader(implementationClassName));
-            }
-
-            Thread.currentThread().setContextClassLoader(detectedClassLoaderForType);
-            final Class<?> desiredClass = rawClass.asSubclass(typeDefinition);
-            return typeDefinition.cast(desiredClass.newInstance());
-        } finally {
-            Thread.currentThread().setContextClassLoader(originalClassLoader);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/Peer.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/Peer.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/Peer.java
deleted file mode 100644
index 2422fe1..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/Peer.java
+++ /dev/null
@@ -1,107 +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.remote;
-
-import java.io.IOException;
-import java.net.URI;
-
-import org.apache.nifi.remote.protocol.CommunicationsSession;
-
-public class Peer {
-
-    private final CommunicationsSession commsSession;
-    private final String url;
-    private final String host;
-    private long penalizationExpiration = 0L;
-    private boolean closed = false;
-
-    public Peer(final CommunicationsSession commsSession, final String url) {
-        this.commsSession = commsSession;
-        this.url = url;
-
-        try {
-            this.host = new URI(url).getHost();
-        } catch (final Exception e) {
-            throw new IllegalArgumentException("Invalid URL: " + url);
-        }
-    }
-
-    public String getUrl() {
-        return url;
-    }
-
-    public CommunicationsSession getCommunicationsSession() {
-        return commsSession;
-    }
-
-    public void close() throws IOException {
-        this.closed = true;
-
-        // TODO: Consume the InputStream so that it doesn't linger on the Peer's outgoing socket buffer
-        commsSession.close();
-    }
-
-    public void penalize(final long millis) {
-        penalizationExpiration = Math.max(penalizationExpiration, System.currentTimeMillis() + millis);
-    }
-
-    public boolean isPenalized() {
-        return penalizationExpiration > System.currentTimeMillis();
-    }
-
-    public boolean isClosed() {
-        return closed;
-    }
-
-    public String getHost() {
-        return host;
-    }
-
-    @Override
-    public int hashCode() {
-        return 8320 + url.hashCode();
-    }
-
-    @Override
-    public boolean equals(final Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (obj == this) {
-            return true;
-        }
-        if (!(obj instanceof Peer)) {
-            return false;
-        }
-
-        final Peer other = (Peer) obj;
-        return this.url.equals(other.url);
-    }
-
-    @Override
-    public String toString() {
-        final StringBuilder sb = new StringBuilder();
-        sb.append("Peer[url=").append(url);
-        if (closed) {
-            sb.append(",CLOSED");
-        } else if (isPenalized()) {
-            sb.append(",PENALIZED");
-        }
-        sb.append("]");
-        return sb.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PeerStatus.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PeerStatus.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PeerStatus.java
deleted file mode 100644
index d1cb076..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PeerStatus.java
+++ /dev/null
@@ -1,72 +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.remote;
-
-public class PeerStatus {
-
-    private final String hostname;
-    private final int port;
-    private final boolean secure;
-    private final int numFlowFiles;
-
-    public PeerStatus(final String hostname, final int port, final boolean secure, final int numFlowFiles) {
-        this.hostname = hostname;
-        this.port = port;
-        this.secure = secure;
-        this.numFlowFiles = numFlowFiles;
-    }
-
-    public String getHostname() {
-        return hostname;
-    }
-
-    public int getPort() {
-        return port;
-    }
-
-    public boolean isSecure() {
-        return secure;
-    }
-
-    public int getFlowFileCount() {
-        return numFlowFiles;
-    }
-
-    @Override
-    public String toString() {
-        return "PeerStatus[hostname=" + hostname + ",port=" + port + ",secure=" + secure + ",flowFileCount=" + numFlowFiles + "]";
-    }
-
-    @Override
-    public int hashCode() {
-        return 9824372 + hostname.hashCode() + port;
-    }
-
-    @Override
-    public boolean equals(final Object obj) {
-        if (obj == null) {
-            return false;
-        }
-
-        if (!(obj instanceof PeerStatus)) {
-            return false;
-        }
-
-        final PeerStatus other = (PeerStatus) obj;
-        return port == other.port && hostname.equals(other.hostname);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PortAuthorizationResult.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PortAuthorizationResult.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PortAuthorizationResult.java
deleted file mode 100644
index 8f2603a..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PortAuthorizationResult.java
+++ /dev/null
@@ -1,25 +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.remote;
-
-public interface PortAuthorizationResult {
-
-    boolean isAuthorized();
-
-    String getExplanation();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteAuthorizationState.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteAuthorizationState.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteAuthorizationState.java
deleted file mode 100644
index 12a3d33..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteAuthorizationState.java
+++ /dev/null
@@ -1,27 +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.remote;
-
-/**
- *
- */
-public enum RemoteAuthorizationState {
-
-    UNKNOWN,
-    UNAUTHORIZED,
-    AUTHORIZED;
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java
deleted file mode 100644
index d4ad374..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java
+++ /dev/null
@@ -1,35 +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.remote;
-
-import org.apache.nifi.connectable.Port;
-import org.apache.nifi.groups.RemoteProcessGroup;
-
-public interface RemoteGroupPort extends Port {
-
-    RemoteProcessGroup getRemoteProcessGroup();
-
-    TransferDirection getTransferDirection();
-
-    boolean isUseCompression();
-
-    void setUseCompression(boolean useCompression);
-
-    boolean getTargetExists();
-
-    boolean isTargetRunning();
-}


[10/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java
deleted file mode 100644
index 630631f..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java
+++ /dev/null
@@ -1,39 +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.cluster;
-
-import javax.xml.bind.annotation.adapters.XmlAdapter;
-
-public class NodeInformationAdapter extends XmlAdapter<AdaptedNodeInformation, NodeInformation> {
-
-    @Override
-    public NodeInformation unmarshal(final AdaptedNodeInformation adapted) throws Exception {
-        return new NodeInformation(adapted.getHostname(), adapted.getSiteToSitePort(), adapted.getApiPort(), adapted.isSiteToSiteSecure(), adapted.getTotalFlowFiles());
-    }
-
-    @Override
-    public AdaptedNodeInformation marshal(final NodeInformation nodeInformation) throws Exception {
-        final AdaptedNodeInformation adapted = new AdaptedNodeInformation();
-        adapted.setHostname(nodeInformation.getHostname());
-        adapted.setSiteToSitePort(nodeInformation.getSiteToSitePort());
-        adapted.setApiPort(nodeInformation.getAPIPort());
-        adapted.setSiteToSiteSecure(nodeInformation.isSiteToSiteSecure());
-        adapted.setTotalFlowFiles(nodeInformation.getTotalFlowFiles());
-        return adapted;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/protocol/DataFlow.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/protocol/DataFlow.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/protocol/DataFlow.java
deleted file mode 100644
index 57c1c30..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/protocol/DataFlow.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.cluster.protocol;
-
-public interface DataFlow {
-
-    /**
-     * @return the raw byte array of the flow
-     */
-    public byte[] getFlow();
-
-    /**
-     * @return the raw byte array of the templates
-     */
-    public byte[] getTemplates();
-
-    /**
-     * @return the raw byte array of the snippets
-     */
-    public byte[] getSnippets();
-
-    /**
-     * @return true if processors should be automatically started at application
-     * startup; false otherwise
-     */
-    public boolean isAutoStartProcessors();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Connectable.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Connectable.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Connectable.java
deleted file mode 100644
index 3d5c75d..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Connectable.java
+++ /dev/null
@@ -1,291 +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.connectable;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.controller.Triggerable;
-import org.apache.nifi.groups.ProcessGroup;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-
-/**
- * Represents a connectable component to which or from which data can flow.
- */
-public interface Connectable extends Triggerable {
-
-    /**
-     * @return the unique identifier for this <code>Connectable</code>
-     */
-    String getIdentifier();
-
-    /**
-     * @return a Collection of all relationships for this Connectable
-     */
-    Collection<Relationship> getRelationships();
-
-    /**
-     * Returns the ProcessorRelationship whose name is given
-     *
-     * @param relationshipName
-     * @return a ProcessorRelationship whose name is given, or <code>null</code>
-     * if none exists
-     */
-    Relationship getRelationship(String relationshipName);
-
-    /**
-     * Adds the given connection to this Connectable.
-     *
-     * @param connection the connection to add
-     * @throws NullPointerException if the argument is null
-     * @throws IllegalArgumentException if the given Connection is not valid
-     */
-    void addConnection(Connection connection) throws IllegalArgumentException;
-
-    /**
-     * @return true if the Connectable is the destination of any other
-     * Connectable, false otherwise.
-     */
-    boolean hasIncomingConnection();
-
-    /**
-     *
-     * @param connection
-     * @throws IllegalStateException if the given Connection is not registered
-     * to <code>this</code>.
-     */
-    void removeConnection(Connection connection) throws IllegalStateException;
-
-    /**
-     * Updates any internal state that depends on the given connection. The
-     * given connection will share the same ID as the old connection.
-     *
-     * @param newConnection
-     * @throws IllegalStateException
-     */
-    void updateConnection(Connection newConnection) throws IllegalStateException;
-
-    /**
-     * @return a <code>Set</code> of all <code>Connection</code>s for which this
-     * <code>Connectable</code> is the destination
-     */
-    List<Connection> getIncomingConnections();
-
-    /**
-     * @return a <code>Set</code> of all <code>Connection</code>s for which this
-     * <code>Connectable</code> is the source; if no connections exist, will
-     * return an empty Collection. Guaranteed not null.
-     */
-    Set<Connection> getConnections();
-
-    /**
-     * @param relationship
-     * @return a <code>Set</code> of all <code>Connection</code>s that contain
-     * the given relationship for which this <code>Connectable</code> is the
-     * source
-     */
-    Set<Connection> getConnections(Relationship relationship);
-
-    /**
-     * Returns the position on the graph where this Connectable is located
-     *
-     * @return
-     */
-    Position getPosition();
-
-    /**
-     * Updates this component's position on the graph
-     *
-     * @param position
-     */
-    void setPosition(Position position);
-
-    /**
-     * @return the name of this Connectable
-     */
-    String getName();
-
-    /**
-     * Sets the name of this Connectable so that its name will be visible on the
-     * UI
-     * @param name
-     */
-    void setName(String name);
-
-    /**
-     * @return the comments of this Connectable
-     */
-    String getComments();
-
-    /**
-     * Sets the comments of this Connectable.
-     * @param comments
-     */
-    void setComments(String comments);
-
-    /**
-     * If true,
-     * {@link #onTrigger(nifi.processor.ProcessContext, nifi.processor.ProcessSessionFactory)}
-     * should be called even when this Connectable has no FlowFiles queued for
-     * processing
-     *
-     * @return
-     */
-    boolean isTriggerWhenEmpty();
-
-    /**
-     * Returns the ProcessGroup to which this <code>Connectable</code> belongs
-     *
-     * @return
-     */
-    ProcessGroup getProcessGroup();
-
-    /**
-     * Sets the new ProcessGroup to which this <code>Connectable</code> belongs
-     *
-     * @param group
-     */
-    void setProcessGroup(ProcessGroup group);
-
-    /**
-     *
-     * @param relationship the relationship
-     * @return true indicates flow files transferred to the given relationship
-     * should be terminated if the relationship is not connected to another
-     * FlowFileConsumer; false indicates they will not be terminated and the
-     * processor will not be valid until specified
-     */
-    boolean isAutoTerminated(Relationship relationship);
-
-    /**
-     * Indicates whether flow file content made by this connectable must be
-     * persisted
-     *
-     * @return 
-     */
-    boolean isLossTolerant();
-
-    /**
-     * @param lossTolerant
-     */
-    void setLossTolerant(boolean lossTolerant);
-
-    /**
-     * @return the type of the Connectable
-     */
-    ConnectableType getConnectableType();
-
-    /**
-     * Returns the any validation errors for this connectable.
-     *
-     * @return
-     */
-    Collection<ValidationResult> getValidationErrors();
-
-    /**
-     * Returns the amount of time for which a FlowFile should be penalized when
-     * {@link ProcessSession#penalize(nifi.flowfile.FlowFile)} is called
-     *
-     * @param timeUnit
-     * @return
-     */
-    long getPenalizationPeriod(final TimeUnit timeUnit);
-
-    /**
-     * Returns a string representation for which a FlowFile should be penalized
-     * when {@link ProcessSession#penalize(nifi.flowfile.FlowFile)} is called
-     *
-     * @return
-     */
-    String getPenalizationPeriod();
-
-    /**
-     * @param timeUnit determines the unit of time to represent the yield
-     * period.
-     * @return
-     */
-    long getYieldPeriod(TimeUnit timeUnit);
-
-    /**
-     * returns the string representation for this Connectable's configured yield
-     * period
-     *
-     * @return
-     */
-    String getYieldPeriod();
-
-    /**
-     * Updates the amount of time that this Connectable should avoid being
-     * scheduled when the processor calls
-     * {@link nifi.processor.ProcessContext#yield() ProcessContext.yield()}
-     *
-     * @param yieldPeriod
-     */
-    void setYieldPeriod(String yieldPeriod);
-
-    /**
-     * Updates the amount of time that this Connectable will penalize FlowFiles
-     * when {@link ProcessSession#penalize(nifi.flowfile.FlowFile)} is called
-     * @param penalizationPeriod
-     */
-    void setPenalizationPeriod(String penalizationPeriod);
-
-    /**
-     * Causes the processor not to be scheduled for some period of time. This
-     * duration can be obtained and set via the
-     * {@link #getYieldPeriod(TimeUnit)} and
-     * {@link #setYieldPeriod(long, TimeUnit)} methods.
-     */
-    void yield();
-
-    /**
-     * Returns the time in milliseconds since Epoch at which this Connectable
-     * should no longer yield its threads
-     *
-     * @return
-     */
-    long getYieldExpiration();
-
-    /**
-     * Specifies whether or not this component is considered side-effect free,
-     * with respect to external systems.
-     *
-     * @return
-     */
-    boolean isSideEffectFree();
-
-    void verifyCanDelete() throws IllegalStateException;
-
-    void verifyCanDelete(boolean ignoreConnections) throws IllegalStateException;
-
-    void verifyCanStart() throws IllegalStateException;
-
-    void verifyCanStop() throws IllegalStateException;
-
-    void verifyCanUpdate() throws IllegalStateException;
-
-    void verifyCanEnable() throws IllegalStateException;
-
-    void verifyCanDisable() throws IllegalStateException;
-
-    SchedulingStrategy getSchedulingStrategy();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/ConnectableType.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/ConnectableType.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/ConnectableType.java
deleted file mode 100644
index 0334bfb..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/ConnectableType.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.connectable;
-
-import javax.xml.bind.annotation.XmlEnum;
-
-@XmlEnum
-public enum ConnectableType {
-
-    PROCESSOR,
-    /**
-     * Port that lives within an RemoteProcessGroup and is used to send data to
-     * remote NiFi instances
-     */
-    REMOTE_INPUT_PORT,
-    /**
-     * Port that lives within a RemoteProcessGroup and is used to receive data
-     * from remote NiFi instances
-     */
-    REMOTE_OUTPUT_PORT,
-    /**
-     * Root Group Input Ports and Local Input Ports
-     */
-    INPUT_PORT,
-    /**
-     * Root Group Output Ports and Local Output Ports
-     */
-    OUTPUT_PORT,
-    FUNNEL
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Connection.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Connection.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Connection.java
deleted file mode 100644
index 0a0089d..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Connection.java
+++ /dev/null
@@ -1,78 +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.connectable;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.nifi.controller.FlowFileQueue;
-import org.apache.nifi.controller.repository.FlowFileRecord;
-import org.apache.nifi.groups.ProcessGroup;
-import org.apache.nifi.processor.FlowFileFilter;
-import org.apache.nifi.processor.Relationship;
-
-public interface Connection {
-
-    void enqueue(FlowFileRecord flowFile);
-
-    void enqueue(Collection<FlowFileRecord> flowFiles);
-
-    Connectable getDestination();
-
-    Collection<Relationship> getRelationships();
-
-    FlowFileQueue getFlowFileQueue();
-
-    String getIdentifier();
-
-    String getName();
-
-    void setName(String name);
-
-    void setBendPoints(List<Position> position);
-
-    List<Position> getBendPoints();
-
-    int getLabelIndex();
-
-    void setLabelIndex(int labelIndex);
-
-    long getZIndex();
-
-    void setZIndex(long zIndex);
-
-    Connectable getSource();
-
-    void setRelationships(Collection<Relationship> newRelationships);
-
-    void setDestination(final Connectable newDestination);
-
-    void setProcessGroup(ProcessGroup processGroup);
-
-    ProcessGroup getProcessGroup();
-
-    void lock();
-
-    void unlock();
-
-    List<FlowFileRecord> poll(FlowFileFilter filter, Set<FlowFileRecord> expiredRecords);
-
-    void verifyCanUpdate() throws IllegalStateException;
-
-    void verifyCanDelete() throws IllegalStateException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Funnel.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Funnel.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Funnel.java
deleted file mode 100644
index cceca8f..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Funnel.java
+++ /dev/null
@@ -1,24 +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.connectable;
-
-import org.apache.nifi.controller.ScheduledState;
-
-public interface Funnel extends Connectable {
-
-    void setScheduledState(ScheduledState scheduledState);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Port.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Port.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Port.java
deleted file mode 100644
index 907dd92..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Port.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.connectable;
-
-public interface Port extends Connectable {
-
-    void shutdown();
-
-    boolean isValid();
-
-    /**
-     * <p>
-     * This method is called just before a Port is scheduled to run, giving the
-     * Port a chance to initialize any resources needed.</p>
-     */
-    void onSchedulingStart();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Position.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Position.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Position.java
deleted file mode 100644
index 75d04f5..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Position.java
+++ /dev/null
@@ -1,36 +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.connectable;
-
-public class Position {
-
-    private final double x;
-    private final double y;
-
-    public Position(final double x, final double y) {
-        this.x = x;
-        this.y = y;
-    }
-
-    public double getX() {
-        return x;
-    }
-
-    public double getY() {
-        return y;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Size.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Size.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Size.java
deleted file mode 100644
index cea13d2..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Size.java
+++ /dev/null
@@ -1,36 +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.connectable;
-
-public class Size {
-
-    private final double width;
-    private final double height;
-
-    public Size(final double width, final double height) {
-        this.width = width;
-        this.height = height;
-    }
-
-    public double getWidth() {
-        return width;
-    }
-
-    public double getHeight() {
-        return height;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java
deleted file mode 100644
index ef4b72a..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java
+++ /dev/null
@@ -1,280 +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;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-import org.apache.nifi.components.ConfigurableComponent;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.controller.service.ControllerServiceNode;
-import org.apache.nifi.controller.service.ControllerServiceProvider;
-import org.apache.nifi.nar.NarCloseable;
-
-public abstract class AbstractConfiguredComponent implements ConfigurableComponent, ConfiguredComponent {
-
-    private final String id;
-    private final ConfigurableComponent component;
-    private final ValidationContextFactory validationContextFactory;
-    private final ControllerServiceProvider serviceProvider;
-
-    private final AtomicReference<String> name = new AtomicReference<>();
-    private final AtomicReference<String> annotationData = new AtomicReference<>();
-
-    private final Lock lock = new ReentrantLock();
-    private final ConcurrentMap<PropertyDescriptor, String> properties = new ConcurrentHashMap<>();
-
-    public AbstractConfiguredComponent(final ConfigurableComponent component, final String id,
-            final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider) {
-        this.id = id;
-        this.component = component;
-        this.validationContextFactory = validationContextFactory;
-        this.serviceProvider = serviceProvider;
-    }
-
-    @Override
-    public String getIdentifier() {
-        return id;
-    }
-
-    @Override
-    public String getName() {
-        return name.get();
-    }
-
-    @Override
-    public void setName(final String name) {
-        this.name.set(Objects.requireNonNull(name).intern());
-    }
-
-    @Override
-    public String getAnnotationData() {
-        return annotationData.get();
-    }
-
-    @Override
-    public void setAnnotationData(final String data) {
-        annotationData.set(data);
-    }
-
-    @Override
-    public void setProperty(final String name, final String value) {
-        if (null == name || null == value) {
-            throw new IllegalArgumentException();
-        }
-
-        lock.lock();
-        try {
-            verifyModifiable();
-
-            try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                final PropertyDescriptor descriptor = component.getPropertyDescriptor(name);
-
-                final String oldValue = properties.put(descriptor, value);
-                if (!value.equals(oldValue)) {
-
-                    if (descriptor.getControllerServiceDefinition() != null) {
-                        if (oldValue != null) {
-                            final ControllerServiceNode oldNode = serviceProvider.getControllerServiceNode(oldValue);
-                            if (oldNode != null) {
-                                oldNode.removeReference(this);
-                            }
-                        }
-
-                        final ControllerServiceNode newNode = serviceProvider.getControllerServiceNode(value);
-                        if (newNode != null) {
-                            newNode.addReference(this);
-                        }
-                    }
-
-                    try {
-                        component.onPropertyModified(descriptor, oldValue, value);
-                    } catch (final Throwable t) {
-                        // nothing really to do here...
-                    }
-                }
-            }
-        } finally {
-            lock.unlock();
-        }
-    }
-
-    /**
-     * Removes the property and value for the given property name if a
-     * descriptor and value exists for the given name. If the property is
-     * optional its value might be reset to default or will be removed entirely
-     * if was a dynamic property.
-     *
-     * @param name the property to remove
-     * @return true if removed; false otherwise
-     * @throws java.lang.IllegalArgumentException if the name is null
-     */
-    @Override
-    public boolean removeProperty(final String name) {
-        if (null == name) {
-            throw new IllegalArgumentException();
-        }
-
-        lock.lock();
-        try {
-            verifyModifiable();
-
-            try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                final PropertyDescriptor descriptor = component.getPropertyDescriptor(name);
-                String value = null;
-                if (!descriptor.isRequired() && (value = properties.remove(descriptor)) != null) {
-                    component.onPropertyModified(descriptor, value, null);
-                    return true;
-                }
-            }
-        } finally {
-            lock.unlock();
-        }
-        return false;
-    }
-
-    @Override
-    public Map<PropertyDescriptor, String> getProperties() {
-        try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-            final List<PropertyDescriptor> supported = component.getPropertyDescriptors();
-            if (supported == null || supported.isEmpty()) {
-                return Collections.unmodifiableMap(properties);
-            } else {
-                final Map<PropertyDescriptor, String> props = new LinkedHashMap<>();
-                for (final PropertyDescriptor descriptor : supported) {
-                    props.put(descriptor, null);
-                }
-                props.putAll(properties);
-                return props;
-            }
-        }
-    }
-
-    @Override
-    public String getProperty(final PropertyDescriptor property) {
-        return properties.get(property);
-    }
-
-    @Override
-    public int hashCode() {
-        return 273171 * id.hashCode();
-    }
-
-    @Override
-    public boolean equals(final Object obj) {
-        if (obj == this) {
-            return true;
-        }
-        if (obj == null) {
-            return false;
-        }
-
-        if (!(obj instanceof ConfiguredComponent)) {
-            return false;
-        }
-
-        final ConfiguredComponent other = (ConfiguredComponent) obj;
-        return id.equals(other.getIdentifier());
-    }
-
-    @Override
-    public String toString() {
-        try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-            return component.toString();
-        }
-    }
-
-    @Override
-    public Collection<ValidationResult> validate(final ValidationContext context) {
-        try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-            return component.validate(context);
-        }
-    }
-
-    @Override
-    public PropertyDescriptor getPropertyDescriptor(final String name) {
-        try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-            return component.getPropertyDescriptor(name);
-        }
-    }
-
-    @Override
-    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
-        try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-            component.onPropertyModified(descriptor, oldValue, newValue);
-        }
-    }
-
-    @Override
-    public List<PropertyDescriptor> getPropertyDescriptors() {
-        try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-            return component.getPropertyDescriptors();
-        }
-    }
-
-    @Override
-    public boolean isValid() {
-        final Collection<ValidationResult> validationResults = validate(validationContextFactory.newValidationContext(getProperties(), getAnnotationData()));
-        for (final ValidationResult result : validationResults) {
-            if (!result.isValid()) {
-                return false;
-            }
-        }
-
-        return true;
-    }
-
-    @Override
-    public Collection<ValidationResult> getValidationErrors() {
-        final List<ValidationResult> results = new ArrayList<>();
-        lock.lock();
-        try {
-            final ValidationContext validationContext = validationContextFactory.newValidationContext(getProperties(), getAnnotationData());
-
-            final Collection<ValidationResult> validationResults;
-            try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                validationResults = component.validate(validationContext);
-            }
-
-            for (final ValidationResult result : validationResults) {
-                if (!result.isValid()) {
-                    results.add(result);
-                }
-            }
-        } catch (final Throwable t) {
-            results.add(new ValidationResult.Builder().explanation("Failed to run validation due to " + t.toString()).valid(false).build());
-        } finally {
-            lock.unlock();
-        }
-        return results;
-    }
-
-    public abstract void verifyModifiable() throws IllegalStateException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java
deleted file mode 100644
index e1d2dd4..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java
+++ /dev/null
@@ -1,636 +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;
-
-import static java.util.Objects.requireNonNull;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-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.components.ValidationResult;
-import org.apache.nifi.connectable.Connectable;
-import org.apache.nifi.connectable.ConnectableType;
-import org.apache.nifi.connectable.Connection;
-import org.apache.nifi.connectable.Port;
-import org.apache.nifi.connectable.Position;
-import org.apache.nifi.groups.ProcessGroup;
-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.util.FormatUtils;
-
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-public abstract class AbstractPort implements Port {
-
-    public static final Relationship PORT_RELATIONSHIP = new Relationship.Builder()
-            .description("The relationship through which all Flow Files are transferred")
-            .name("")
-            .build();
-
-    public static final long MINIMUM_PENALIZATION_MILLIS = 0L;
-    public static final TimeUnit DEFAULT_TIME_UNIT = TimeUnit.MILLISECONDS;
-
-    public static final long MINIMUM_YIELD_MILLIS = 0L;
-    public static final long DEFAULT_YIELD_PERIOD = 10000L;
-    public static final TimeUnit DEFAULT_YIELD_TIME_UNIT = TimeUnit.MILLISECONDS;
-
-    private final List<Relationship> relationships;
-
-    private final String id;
-    private final ConnectableType type;
-    private final AtomicReference<String> name;
-    private final AtomicReference<Position> position;
-    private final AtomicReference<String> comments;
-    private final AtomicReference<ProcessGroup> processGroup;
-    private final AtomicBoolean lossTolerant;
-    private final AtomicReference<ScheduledState> scheduledState;
-    private final AtomicInteger concurrentTaskCount;
-    private final AtomicReference<String> penalizationPeriod;
-    private final AtomicReference<String> yieldPeriod;
-    private final AtomicReference<String> schedulingPeriod;
-    private final AtomicLong schedulingNanos;
-    private final AtomicLong yieldExpiration;
-    private final ProcessScheduler processScheduler;
-
-    private final Set<Connection> outgoingConnections;
-    private final List<Connection> incomingConnections;
-
-    private final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
-    private final Lock readLock = rwLock.readLock();
-    private final Lock writeLock = rwLock.writeLock();
-
-    public AbstractPort(final String id, final String name, final ProcessGroup processGroup, final ConnectableType type, final ProcessScheduler scheduler) {
-        this.id = requireNonNull(id);
-        this.name = new AtomicReference<>(requireNonNull(name));
-        position = new AtomicReference<>(new Position(0D, 0D));
-        outgoingConnections = new HashSet<>();
-        incomingConnections = new ArrayList<>();
-        comments = new AtomicReference<>();
-        lossTolerant = new AtomicBoolean(false);
-        concurrentTaskCount = new AtomicInteger(1);
-        processScheduler = scheduler;
-
-        final List<Relationship> relationshipList = new ArrayList<>();
-        relationshipList.add(PORT_RELATIONSHIP);
-        relationships = Collections.unmodifiableList(relationshipList);
-        this.processGroup = new AtomicReference<>(processGroup);
-        this.type = type;
-        penalizationPeriod = new AtomicReference<>("30 sec");
-        yieldPeriod = new AtomicReference<>("1 sec");
-        yieldExpiration = new AtomicLong(0L);
-        schedulingPeriod = new AtomicReference<>("0 millis");
-        schedulingNanos = new AtomicLong(30000);
-        scheduledState = new AtomicReference<>(ScheduledState.STOPPED);
-    }
-
-    @Override
-    public String getIdentifier() {
-        return id;
-    }
-
-    @Override
-    public String getName() {
-        return name.get();
-    }
-
-    @Override
-    public void setName(final String name) {
-        if (this.name.get().equals(name)) {
-            return;
-        }
-
-        final ProcessGroup parentGroup = this.processGroup.get();
-        if (getConnectableType() == ConnectableType.INPUT_PORT) {
-            if (parentGroup.getInputPortByName(name) != null) {
-                throw new IllegalStateException("Cannot rename port from " + this.name.get() + " to " + name + " because the ProcessGroup already has an Input Port named " + name);
-            }
-        } else if (getConnectableType() == ConnectableType.OUTPUT_PORT) {
-            if (parentGroup.getOutputPortByName(name) != null) {
-                throw new IllegalStateException("Cannot rename port from " + this.name.get() + " to " + name + " because the ProcessGroup already has an Output Port named " + name);
-            }
-        }
-
-        this.name.set(name);
-    }
-
-    @Override
-    public ProcessGroup getProcessGroup() {
-        return processGroup.get();
-    }
-
-    @Override
-    public void setProcessGroup(final ProcessGroup newGroup) {
-        this.processGroup.set(newGroup);
-    }
-
-    @Override
-    public String getComments() {
-        return comments.get();
-    }
-
-    @Override
-    public void setComments(final String comments) {
-        this.comments.set(comments);
-    }
-
-    @Override
-    public Collection<Relationship> getRelationships() {
-        return relationships;
-    }
-
-    @Override
-    public Relationship getRelationship(final String relationshipName) {
-        if (PORT_RELATIONSHIP.getName().equals(relationshipName)) {
-            return PORT_RELATIONSHIP;
-        }
-        return null;
-    }
-
-    @Override
-    public void addConnection(final Connection connection) throws IllegalArgumentException {
-        writeLock.lock();
-        try {
-            if (!requireNonNull(connection).getSource().equals(this)) {
-                if (connection.getDestination().equals(this)) {
-                    // don't add the connection twice. This may occur if we have a self-loop because we will be told
-                    // to add the connection once because we are the source and again because we are the destination.
-                    if (!incomingConnections.contains(connection)) {
-                        incomingConnections.add(connection);
-                    }
-
-                    return;
-                } else {
-                    throw new IllegalArgumentException("Cannot add a connection to a LocalPort for which the LocalPort is neither the Source nor the Destination");
-                }
-            }
-
-            for (final Relationship relationship : connection.getRelationships()) {
-                if (!relationship.equals(PORT_RELATIONSHIP)) {
-                    throw new IllegalArgumentException("No relationship with name " + relationship + " exists for Local Ports");
-                }
-            }
-
-            // don't add the connection twice. This may occur if we have a self-loop because we will be told
-            // to add the connection once because we are the source and again because we are the destination.
-            if (!outgoingConnections.contains(connection)) {
-                outgoingConnections.add(connection);
-            }
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public boolean hasIncomingConnection() {
-        readLock.lock();
-        try {
-            return !incomingConnections.isEmpty();
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
-        final ProcessSession session = sessionFactory.createSession();
-
-        try {
-            onTrigger(context, session);
-            session.commit();
-        } catch (final ProcessException e) {
-            session.rollback();
-            throw e;
-        } catch (final Throwable t) {
-            session.rollback();
-            throw new RuntimeException(t);
-        }
-    }
-
-    public abstract void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException;
-
-    @Override
-    public void updateConnection(final Connection connection) throws IllegalStateException {
-        if (requireNonNull(connection).getSource().equals(this)) {
-            writeLock.lock();
-            try {
-                if (!outgoingConnections.remove(connection)) {
-                    throw new IllegalStateException("No Connection with ID " + connection.getIdentifier() + " is currently registered with this Port");
-                }
-                outgoingConnections.add(connection);
-            } finally {
-                writeLock.unlock();
-            }
-        } else if (connection.getDestination().equals(this)) {
-            writeLock.lock();
-            try {
-                if (!incomingConnections.remove(connection)) {
-                    throw new IllegalStateException("No Connection with ID " + connection.getIdentifier() + " is currently registered with this Port");
-                }
-                incomingConnections.add(connection);
-            } finally {
-                writeLock.unlock();
-            }
-        } else {
-            throw new IllegalStateException("The given connection is not currently registered for this Port");
-        }
-    }
-
-    @Override
-    public void removeConnection(final Connection connection) throws IllegalArgumentException, IllegalStateException {
-        writeLock.lock();
-        try {
-            if (!requireNonNull(connection).getSource().equals(this)) {
-                final boolean existed = incomingConnections.remove(connection);
-                if (!existed) {
-                    throw new IllegalStateException("The given connection is not currently registered for this Port");
-                }
-                return;
-            }
-
-            if (!canConnectionBeRemoved(connection)) {
-                // TODO: Determine which processors will be broken if connection is removed, rather than just returning a boolean
-                throw new IllegalStateException(connection + " cannot be removed");
-            }
-
-            final boolean removed = outgoingConnections.remove(connection);
-            if (!removed) {
-                throw new IllegalStateException(connection + " is not registered with " + this);
-            }
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * Verify that removing this connection will not prevent this Port from
-     * still being connected via each relationship
-     *
-     * @param connection
-     * @return
-     */
-    private boolean canConnectionBeRemoved(final Connection connection) {
-        final Connectable source = connection.getSource();
-        if (!source.isRunning()) {
-            // we don't have to verify that this Connectable is still connected because it's okay to make
-            // the source invalid since it is not running.
-            return true;
-        }
-
-        for (final Relationship relationship : source.getRelationships()) {
-            if (source.isAutoTerminated(relationship)) {
-                continue;
-            }
-
-            final Set<Connection> connectionsForRelationship = source.getConnections(relationship);
-            if (connectionsForRelationship == null || connectionsForRelationship.isEmpty()) {
-                return false;
-            }
-        }
-
-        return true;
-    }
-
-    @Override
-    public Set<Connection> getConnections() {
-        readLock.lock();
-        try {
-            return Collections.unmodifiableSet(outgoingConnections);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public Set<Connection> getConnections(final Relationship relationship) {
-        readLock.lock();
-        try {
-            if (relationship.equals(PORT_RELATIONSHIP)) {
-                return Collections.unmodifiableSet(outgoingConnections);
-            }
-
-            throw new IllegalArgumentException("No relationship with name " + relationship.getName() + " exists for Local Ports");
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public Position getPosition() {
-        return position.get();
-    }
-
-    @Override
-    public void setPosition(final Position position) {
-        this.position.set(position);
-    }
-
-    @Override
-    public String toString() {
-        return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE).append("name", getName()).append("id", getIdentifier()).toString();
-    }
-
-    @Override
-    public List<Connection> getIncomingConnections() {
-        readLock.lock();
-        try {
-            return Collections.unmodifiableList(incomingConnections);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    /**
-     * Indicates whether or not this Port is valid.
-     *
-     * @return
-     */
-    @Override
-    public abstract boolean isValid();
-
-    @Override
-    public boolean isAutoTerminated(final Relationship relationship) {
-        return false;
-    }
-
-    @Override
-    public boolean isLossTolerant() {
-        return lossTolerant.get();
-    }
-
-    @Override
-    public void setLossTolerant(boolean lossTolerant) {
-        this.lossTolerant.set(lossTolerant);
-    }
-
-    @Override
-    public void setMaxConcurrentTasks(final int taskCount) {
-        if (taskCount < 1) {
-            throw new IllegalArgumentException();
-        }
-        concurrentTaskCount.set(taskCount);
-    }
-
-    /**
-     * @return the number of tasks that may execute concurrently for this
-     * processor
-     */
-    @Override
-    public int getMaxConcurrentTasks() {
-        return concurrentTaskCount.get();
-    }
-
-    /**
-     *
-     */
-    @Override
-    public void shutdown() {
-        scheduledState.set(ScheduledState.STOPPED);
-    }
-
-    @Override
-    public void onSchedulingStart() {
-        scheduledState.set(ScheduledState.RUNNING);
-    }
-
-    public void disable() {
-        final boolean updated = scheduledState.compareAndSet(ScheduledState.STOPPED, ScheduledState.DISABLED);
-        if (!updated) {
-            throw new IllegalStateException("Port cannot be disabled because it is not stopped");
-        }
-    }
-
-    public void enable() {
-        final boolean updated = scheduledState.compareAndSet(ScheduledState.DISABLED, ScheduledState.STOPPED);
-        if (!updated) {
-            throw new IllegalStateException("Port cannot be enabled because it is not disabled");
-        }
-    }
-
-    @Override
-    public boolean isRunning() {
-        return getScheduledState().equals(ScheduledState.RUNNING) || processScheduler.getActiveThreadCount(this) > 0;
-    }
-
-    @Override
-    public ScheduledState getScheduledState() {
-        return scheduledState.get();
-    }
-
-    @Override
-    public ConnectableType getConnectableType() {
-        return type;
-    }
-
-    /**
-     * Updates the amount of time that this processor should avoid being
-     * scheduled when the processor calls
-     * {@link nifi.processor.ProcessContext#yield() ProcessContext.yield()}
-     *
-     * @param yieldPeriod
-     */
-    @Override
-    public void setYieldPeriod(final String yieldPeriod) {
-        final long yieldMillis = FormatUtils.getTimeDuration(requireNonNull(yieldPeriod), TimeUnit.MILLISECONDS);
-        if (yieldMillis < 0) {
-            throw new IllegalArgumentException("Yield duration must be positive");
-        }
-        this.yieldPeriod.set(yieldPeriod);
-    }
-
-    /**
-     * @param schedulingPeriod
-     */
-    @Override
-    public void setScheduldingPeriod(final String schedulingPeriod) {
-        final long schedulingNanos = FormatUtils.getTimeDuration(requireNonNull(schedulingPeriod), TimeUnit.NANOSECONDS);
-        if (schedulingNanos < 0) {
-            throw new IllegalArgumentException("Scheduling Period must be positive");
-        }
-
-        this.schedulingPeriod.set(schedulingPeriod);
-        this.schedulingNanos.set(Math.max(MINIMUM_SCHEDULING_NANOS, schedulingNanos));
-    }
-
-    @Override
-    public long getPenalizationPeriod(final TimeUnit timeUnit) {
-        return FormatUtils.getTimeDuration(getPenalizationPeriod(), timeUnit == null ? DEFAULT_TIME_UNIT : timeUnit);
-    }
-
-    @Override
-    public String getPenalizationPeriod() {
-        return penalizationPeriod.get();
-    }
-
-    /**
-     * Causes the processor not to be scheduled for some period of time. This
-     * duration can be obtained and set via the
-     * {@link #getYieldPeriod(TimeUnit)} and
-     * {@link #setYieldPeriod(long, TimeUnit)} methods.
-     */
-    @Override
-    public void yield() {
-        final long yieldMillis = getYieldPeriod(TimeUnit.MILLISECONDS);
-        yieldExpiration.set(Math.max(yieldExpiration.get(), System.currentTimeMillis() + yieldMillis));
-    }
-
-    @Override
-    public long getYieldExpiration() {
-        return yieldExpiration.get();
-    }
-
-    @Override
-    public long getSchedulingPeriod(final TimeUnit timeUnit) {
-        return timeUnit.convert(schedulingNanos.get(), TimeUnit.NANOSECONDS);
-    }
-
-    @Override
-    public String getSchedulingPeriod() {
-        return schedulingPeriod.get();
-    }
-
-    @Override
-    public void setPenalizationPeriod(final String penalizationPeriod) {
-        this.penalizationPeriod.set(penalizationPeriod);
-    }
-
-    @Override
-    public String getYieldPeriod() {
-        return yieldPeriod.get();
-    }
-
-    @Override
-    public long getYieldPeriod(final TimeUnit timeUnit) {
-        return FormatUtils.getTimeDuration(getYieldPeriod(), timeUnit == null ? DEFAULT_TIME_UNIT : timeUnit);
-    }
-
-    @Override
-    public void verifyCanDelete() throws IllegalStateException {
-        verifyCanDelete(false);
-    }
-
-    @Override
-    public void verifyCanDelete(final boolean ignoreConnections) {
-        readLock.lock();
-        try {
-            if (isRunning()) {
-                throw new IllegalStateException(this + " is running");
-            }
-
-            if (!ignoreConnections) {
-                for (final Connection connection : outgoingConnections) {
-                    connection.verifyCanDelete();
-                }
-
-                for (final Connection connection : incomingConnections) {
-                    if (connection.getSource().equals(this)) {
-                        connection.verifyCanDelete();
-                    } else {
-                        throw new IllegalStateException(this + " is the destination of another component");
-                    }
-                }
-            }
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public void verifyCanStart() {
-        readLock.lock();
-        try {
-            if (scheduledState.get() != ScheduledState.STOPPED) {
-                throw new IllegalStateException(this + " is not stopped");
-            }
-            verifyNoActiveThreads();
-
-            final Collection<ValidationResult> validationResults = getValidationErrors();
-            if (!validationResults.isEmpty()) {
-                throw new IllegalStateException(this + " is not in a valid state: " + validationResults.iterator().next().getExplanation());
-            }
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public void verifyCanStop() {
-        if (getScheduledState() != ScheduledState.RUNNING) {
-            throw new IllegalStateException(this + " is not scheduled to run");
-        }
-    }
-
-    @Override
-    public void verifyCanUpdate() {
-        readLock.lock();
-        try {
-            if (isRunning()) {
-                throw new IllegalStateException(this + " is not stopped");
-            }
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public void verifyCanEnable() {
-        readLock.lock();
-        try {
-            if (getScheduledState() != ScheduledState.DISABLED) {
-                throw new IllegalStateException(this + " is not disabled");
-            }
-
-            verifyNoActiveThreads();
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public void verifyCanDisable() {
-        readLock.lock();
-        try {
-            if (getScheduledState() != ScheduledState.STOPPED) {
-                throw new IllegalStateException(this + " is not stopped");
-            }
-            verifyNoActiveThreads();
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    private void verifyNoActiveThreads() throws IllegalStateException {
-        final int threadCount = processScheduler.getActiveThreadCount(this);
-        if (threadCount > 0) {
-            throw new IllegalStateException(this + " has " + threadCount + " threads still active");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Availability.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Availability.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Availability.java
deleted file mode 100644
index 38df6f7..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Availability.java
+++ /dev/null
@@ -1,24 +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;
-
-public enum Availability {
-
-    CLUSTER_MANAGER_ONLY,
-    NODE_ONLY,
-    BOTH;
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java
deleted file mode 100644
index 5b95524..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.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.controller;
-
-import java.util.Collection;
-import java.util.Map;
-
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.ValidationResult;
-
-public interface ConfiguredComponent {
-
-    public String getIdentifier();
-
-    public String getName();
-
-    public void setName(String name);
-
-    public String getAnnotationData();
-
-    public void setAnnotationData(String data);
-
-    public void setProperty(String name, String value);
-
-    /**
-     * Removes the property and value for the given property name if a
-     * descriptor and value exists for the given name. If the property is
-     * optional its value might be reset to default or will be removed entirely
-     * if was a dynamic property.
-     *
-     * @param name the property to remove
-     * @return true if removed; false otherwise
-     * @throws java.lang.IllegalArgumentException if the name is null
-     */
-    public boolean removeProperty(String name);
-
-    public Map<PropertyDescriptor, String> getProperties();
-
-    public String getProperty(final PropertyDescriptor property);
-
-    boolean isValid();
-
-    /**
-     * Returns the any validation errors for this connectable.
-     *
-     * @return
-     */
-    Collection<ValidationResult> getValidationErrors();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ContentAvailability.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ContentAvailability.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ContentAvailability.java
deleted file mode 100644
index eee878e..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ContentAvailability.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.controller;
-
-/**
- * Provides information about whether or not the data referenced in a Provenance
- * Event can be replayed or downloaded
- */
-public interface ContentAvailability {
-
-    /**
-     * Returns a boolean indicating whether or not the Input content is
-     * available
-     *
-     * @return
-     */
-    boolean isInputAvailable();
-
-    /**
-     * Returns a boolean indicating whether or not the Output content is
-     * available
-     *
-     * @return
-     */
-    boolean isOutputAvailable();
-
-    /**
-     * Returns <code>true</code> if the Input content is the same as the Output
-     * content
-     *
-     * @return
-     */
-    boolean isContentSame();
-
-    /**
-     * Returns a boolean indicating whether or not the content is replayable. If
-     * this returns <code>false</code>, the reason that replay is not available
-     * can be determined by calling {@link #getReasonNotReplayable()}.
-     *
-     * @return
-     */
-    boolean isReplayable();
-
-    /**
-     * Returns the reason that the content cannot be replayed, or
-     * <code>null</code> if the content can be replayed.
-     *
-     * @return
-     */
-    String getReasonNotReplayable();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Counter.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Counter.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Counter.java
deleted file mode 100644
index eaa0c48..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Counter.java
+++ /dev/null
@@ -1,32 +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;
-
-public interface Counter {
-
-    void adjust(long delta);
-
-    String getName();
-
-    long getValue();
-
-    String getContext();
-
-    String getIdentifier();
-
-    void reset();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/EventBasedWorker.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/EventBasedWorker.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/EventBasedWorker.java
deleted file mode 100644
index 280f69d..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/EventBasedWorker.java
+++ /dev/null
@@ -1,32 +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;
-
-import org.apache.nifi.connectable.Connectable;
-
-/**
- * Wraps a Connectable object and maintains a count of how many unanswered
- * events have been reported for the Connectable
- */
-public interface EventBasedWorker {
-
-    Connectable getConnectable();
-
-    int incrementEventCount();
-
-    int decrementEventCount();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Heartbeater.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Heartbeater.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Heartbeater.java
deleted file mode 100644
index 1195bc9..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Heartbeater.java
+++ /dev/null
@@ -1,22 +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;
-
-public interface Heartbeater {
-
-    void heartbeat();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java
deleted file mode 100644
index 303f540..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java
+++ /dev/null
@@ -1,146 +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;
-
-import org.apache.nifi.connectable.Connectable;
-import org.apache.nifi.connectable.Funnel;
-import org.apache.nifi.connectable.Port;
-import org.apache.nifi.processor.annotation.OnScheduled;
-import org.apache.nifi.processor.annotation.OnUnscheduled;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-
-public interface ProcessScheduler {
-
-    /**
-     * Shuts down the scheduler, stopping all components
-     */
-    void shutdown();
-
-    /**
-     * Starts scheduling the given processor to run after invoking all methods
-     * on the underlying {@link nifi.processor.Processor FlowFileProcessor} that
-     * are annotated with the {@link OnScheduled} annotation. If the Processor
-     * is already scheduled to run, does nothing.
-     *
-     * @param procNode
-     * @throws IllegalStateException if the Processor is disabled
-     */
-    void startProcessor(ProcessorNode procNode);
-
-    /**
-     * Stops scheduling the given processor to run and invokes all methods on
-     * the underlying {@link nifi.processor.Processor FlowFileProcessor} that
-     * are annotated with the {@link OnUnscheduled} annotation. This does not
-     * interrupt any threads that are currently running within the given
-     * Processor. If the Processor is not scheduled to run, does nothing.
-     * @param procNode
-     */
-    void stopProcessor(ProcessorNode procNode);
-
-    /**
-     * Starts scheduling the given Port to run. If the Port is already scheduled
-     * to run, does nothing.
-     *
-     * @param port
-     *
-     * @throws IllegalStateException if the Port is disabled
-     */
-    void startPort(Port port);
-
-    /**
-     * Stops scheduling the given Port to run. This does not interrupt any
-     * threads that are currently running within the given port. This does not
-     * interrupt any threads that are currently running within the given Port.
-     * If the Port is not scheduled to run, does nothing.
-     *
-     * @param port
-     */
-    void stopPort(Port port);
-
-    /**
-     * Starts scheduling the given Funnel to run. If the funnel is already
-     * scheduled to run, does nothing.
-     *
-     * @param funnel
-     *
-     * @throws IllegalStateException if the Funnel is disabled
-     */
-    void startFunnel(Funnel funnel);
-
-    /**
-     * Stops scheduling the given Funnel to run. This does not interrupt any
-     * threads that are currently running within the given funnel. If the funnel
-     * is not scheduled to run, does nothing.
-     *
-     * @param funnel
-     */
-    void stopFunnel(Funnel funnel);
-
-    void enableFunnel(Funnel funnel);
-
-    void enablePort(Port port);
-
-    void enableProcessor(ProcessorNode procNode);
-
-    void disableFunnel(Funnel funnel);
-
-    void disablePort(Port port);
-
-    void disableProcessor(ProcessorNode procNode);
-
-    /**
-     * Returns the number of threads currently active for the given
-     * <code>Connectable</code>.
-     *
-     * @param scheduled
-     * @return
-     */
-    int getActiveThreadCount(Object scheduled);
-
-    /**
-     * Returns a boolean indicating whether or not the given object is scheduled
-     * to run
-     *
-     * @param scheduled
-     * @return
-     */
-    boolean isScheduled(Object scheduled);
-
-    /**
-     * Registers a relevant event for an Event-Driven worker
-     *
-     * @param worker
-     */
-    void registerEvent(Connectable worker);
-
-    /**
-     * Notifies the ProcessScheduler of how many threads are available to use
-     * for the given {@link SchedulingStrategy}
-     *
-     * @param strategy
-     * @param maxThreadCount
-     */
-    void setMaxThreadCount(SchedulingStrategy strategy, int maxThreadCount);
-
-    /**
-     * Notifies the Scheduler that it should stop scheduling the given component
-     * until its yield duration has expired
-     *
-     * @param procNode
-     */
-    void yield(ProcessorNode procNode);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
deleted file mode 100644
index f6786fa..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
+++ /dev/null
@@ -1,80 +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;
-
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.connectable.Connectable;
-import org.apache.nifi.controller.service.ControllerServiceProvider;
-import org.apache.nifi.logging.LogLevel;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-
-public abstract class ProcessorNode extends AbstractConfiguredComponent implements Connectable {
-
-    public ProcessorNode(final Processor processor, final String id,
-            final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider) {
-        super(processor, id, validationContextFactory, serviceProvider);
-    }
-
-    public abstract boolean isIsolated();
-
-    public abstract boolean isTriggerWhenAnyDestinationAvailable();
-
-    @Override
-    public abstract boolean isSideEffectFree();
-
-    public abstract boolean isTriggeredSerially();
-
-    public abstract boolean isEventDrivenSupported();
-
-    public abstract boolean isHighThroughputSupported();
-
-    @Override
-    public abstract boolean isValid();
-
-    public abstract void setScheduledState(ScheduledState scheduledState);
-
-    public abstract void setBulletinLevel(LogLevel bulletinLevel);
-
-    public abstract LogLevel getBulletinLevel();
-
-    public abstract Processor getProcessor();
-
-    public abstract void yield(long period, TimeUnit timeUnit);
-
-    public abstract void setAutoTerminatedRelationships(Set<Relationship> relationships);
-
-    public abstract Set<Relationship> getAutoTerminatedRelationships();
-
-    public abstract void setSchedulingStrategy(SchedulingStrategy schedulingStrategy);
-
-    @Override
-    public abstract SchedulingStrategy getSchedulingStrategy();
-
-    public abstract void setRunDuration(long duration, TimeUnit timeUnit);
-
-    public abstract long getRunDuration(TimeUnit timeUnit);
-
-    public abstract Map<String, String> getStyle();
-
-    public abstract void setStyle(Map<String, String> style);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java
deleted file mode 100644
index 6b8ede0..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java
+++ /dev/null
@@ -1,56 +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;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.reporting.ReportingContext;
-import org.apache.nifi.reporting.ReportingTask;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-
-public interface ReportingTaskNode extends ConfiguredComponent {
-
-    Availability getAvailability();
-
-    void setAvailability(Availability availability);
-
-    void setSchedulingStrategy(SchedulingStrategy schedulingStrategy);
-
-    SchedulingStrategy getSchedulingStrategy();
-
-    /**
-     * @return a string representation of the time between each scheduling
-     * period
-     */
-    String getSchedulingPeriod();
-
-    long getSchedulingPeriod(TimeUnit timeUnit);
-
-    /**
-     * Updates how often the ReportingTask should be triggered to run
-     * @param schedulingPeriod
-     */
-    void setScheduldingPeriod(String schedulingPeriod);
-
-    ReportingTask getReportingTask();
-
-    ReportingContext getReportingContext();
-
-    ConfigurationContext getConfigurationContext();
-
-    boolean isRunning();
-}


[24/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/SnippetDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/SnippetDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/SnippetDTO.java
deleted file mode 100644
index 2ee1310..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/SnippetDTO.java
+++ /dev/null
@@ -1,239 +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.api.dto;
-
-import java.util.HashSet;
-import java.util.Set;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * The contents of a snippet of a flow.
- */
-@XmlType(name = "snippet")
-public class SnippetDTO {
-
-    private String id;
-    private String uri;
-    private String parentGroupId;
-    private Boolean linked;
-
-    // when specified these are only considered during creation
-    private Set<String> processGroups = new HashSet<>();
-    private Set<String> remoteProcessGroups = new HashSet<>();
-    private Set<String> processors = new HashSet<>();
-    private Set<String> inputPorts = new HashSet<>();
-    private Set<String> outputPorts = new HashSet<>();
-    private Set<String> connections = new HashSet<>();
-    private Set<String> labels = new HashSet<>();
-    private Set<String> funnels = new HashSet<>();
-
-    private FlowSnippetDTO contents;
-
-    /**
-     * The id of this snippet.
-     *
-     * @return
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The uri of this snippet.
-     *
-     * @return
-     */
-    public String getUri() {
-        return uri;
-    }
-
-    public void setUri(String uri) {
-        this.uri = uri;
-    }
-
-    /**
-     * The group id for the components in this snippet.
-     *
-     * @return
-     */
-    public String getParentGroupId() {
-        return parentGroupId;
-    }
-
-    public void setParentGroupId(String parentGroupId) {
-        this.parentGroupId = parentGroupId;
-    }
-
-    /**
-     * Whether or not this snippet is linked to the underlying data flow.
-     *
-     * @return
-     */
-    public Boolean isLinked() {
-        return linked;
-    }
-
-    public void setLinked(Boolean linked) {
-        this.linked = linked;
-    }
-
-    /**
-     * The ids of the connections in this snippet. These ids will be populated
-     * within each response. They can be specified when creating a snippet.
-     * However, once a snippet has been created its contents cannot be modified
-     * (these ids are ignored during update requests).
-     *
-     * @return
-     */
-    public Set<String> getConnections() {
-        return connections;
-    }
-
-    public void setConnections(Set<String> connections) {
-        this.connections = connections;
-    }
-
-    /**
-     * The ids of the funnels in this snippet. These ids will be populated
-     * within each response. They can be specified when creating a snippet.
-     * However, once a snippet has been created its contents cannot be modified
-     * (these ids are ignored during update requests).
-     *
-     * @param funnels
-     */
-    public Set<String> getFunnels() {
-        return funnels;
-    }
-
-    public void setFunnels(Set<String> funnels) {
-        this.funnels = funnels;
-    }
-
-    /**
-     * The ids of the input port in this snippet. These ids will be populated
-     * within each response. They can be specified when creating a snippet.
-     * However, once a snippet has been created its contents cannot be modified
-     * (these ids are ignored during update requests).
-     *
-     * @return
-     */
-    public Set<String> getInputPorts() {
-        return inputPorts;
-    }
-
-    public void setInputPorts(Set<String> inputPorts) {
-        this.inputPorts = inputPorts;
-    }
-
-    /**
-     * The ids of the labels in this snippet. These ids will be populated within
-     * each response. They can be specified when creating a snippet. However,
-     * once a snippet has been created its contents cannot be modified (these
-     * ids are ignored during update requests).
-     *
-     * @return
-     */
-    public Set<String> getLabels() {
-        return labels;
-    }
-
-    public void setLabels(Set<String> labels) {
-        this.labels = labels;
-    }
-
-    /**
-     * The ids of the output ports in this snippet. These ids will be populated
-     * within each response. They can be specified when creating a snippet.
-     * However, once a snippet has been created its contents cannot be modified
-     * (these ids are ignored during update requests).
-     *
-     * @return
-     */
-    public Set<String> getOutputPorts() {
-        return outputPorts;
-    }
-
-    public void setOutputPorts(Set<String> outputPorts) {
-        this.outputPorts = outputPorts;
-    }
-
-    /**
-     * The ids of the process groups in this snippet. These ids will be
-     * populated within each response. They can be specified when creating a
-     * snippet. However, once a snippet has been created its contents cannot be
-     * modified (these ids are ignored during update requests).
-     *
-     * @return
-     */
-    public Set<String> getProcessGroups() {
-        return processGroups;
-    }
-
-    public void setProcessGroups(Set<String> processGroups) {
-        this.processGroups = processGroups;
-    }
-
-    /**
-     * The ids of the processors in this snippet. These ids will be populated
-     * within each response. They can be specified when creating a snippet.
-     * However, once a snippet has been created its contents cannot be modified
-     * (these ids are ignored during update requests).
-     *
-     * @return
-     */
-    public Set<String> getProcessors() {
-        return processors;
-    }
-
-    public void setProcessors(Set<String> processors) {
-        this.processors = processors;
-    }
-
-    /**
-     * The ids of the remote process groups in this snippet. These ids will be
-     * populated within each response. They can be specified when creating a
-     * snippet. However, once a snippet has been created its contents cannot be
-     * modified (these ids are ignored during update requests).
-     *
-     * @return
-     */
-    public Set<String> getRemoteProcessGroups() {
-        return remoteProcessGroups;
-    }
-
-    public void setRemoteProcessGroups(Set<String> remoteProcessGroups) {
-        this.remoteProcessGroups = remoteProcessGroups;
-    }
-
-    /**
-     * The contents of the configuration for this snippet.
-     *
-     * @return
-     */
-    public FlowSnippetDTO getContents() {
-        return contents;
-    }
-
-    public void setContents(FlowSnippetDTO contents) {
-        this.contents = contents;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO.java
deleted file mode 100644
index 0b20852..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO.java
+++ /dev/null
@@ -1,461 +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.api.dto;
-
-import java.util.Date;
-import java.util.Set;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * The diagnostics of the system this NiFi is running on.
- */
-@XmlType(name = "systemDiagnostics")
-public class SystemDiagnosticsDTO {
-
-    private String totalNonHeap;
-    private String usedNonHeap;
-    private String freeNonHeap;
-    private String maxNonHeap;
-    private String nonHeapUtilization;
-
-    private String totalHeap;
-    private String usedHeap;
-    private String freeHeap;
-    private String maxHeap;
-    private String heapUtilization;
-
-    private Integer availableProcessors;
-    private Double processorLoadAverage;
-
-    private Integer totalThreads;
-    private Integer daemonThreads;
-
-    private StorageUsageDTO flowFileRepositoryStorageUsage;
-    private Set<StorageUsageDTO> contentRepositoryStorageUsage;
-    private Set<GarbageCollectionDTO> garbageCollection;
-
-    private Date statsLastRefreshed;
-
-    /* getters / setters */
-    /**
-     * The number of available processors, if supported.
-     *
-     * @return
-     */
-    public Integer getAvailableProcessors() {
-        return availableProcessors;
-    }
-
-    public void setAvailableProcessors(Integer availableProcessors) {
-        this.availableProcessors = availableProcessors;
-    }
-
-    /**
-     * The number of daemon threads.
-     *
-     * @return
-     */
-    public Integer getDaemonThreads() {
-        return daemonThreads;
-    }
-
-    public void setDaemonThreads(Integer daemonThreads) {
-        this.daemonThreads = daemonThreads;
-    }
-
-    /**
-     * The amount of free heap.
-     *
-     * @return
-     */
-    public String getFreeHeap() {
-        return freeHeap;
-    }
-
-    public void setFreeHeap(String freeHeap) {
-        this.freeHeap = freeHeap;
-    }
-
-    /**
-     * The amount of free non-heap.
-     *
-     * @return
-     */
-    public String getFreeNonHeap() {
-        return freeNonHeap;
-    }
-
-    public void setFreeNonHeap(String freeNonHeap) {
-        this.freeNonHeap = freeNonHeap;
-    }
-
-    /**
-     * The max size of the heap.
-     *
-     * @return
-     */
-    public String getMaxHeap() {
-        return maxHeap;
-    }
-
-    public void setMaxHeap(String maxHeap) {
-        this.maxHeap = maxHeap;
-    }
-
-    /**
-     * The max size of the non-heap.
-     *
-     * @return
-     */
-    public String getMaxNonHeap() {
-        return maxNonHeap;
-    }
-
-    public void setMaxNonHeap(String maxNonHeap) {
-        this.maxNonHeap = maxNonHeap;
-    }
-
-    /**
-     * The processor load average, if supported.
-     *
-     * @return
-     */
-    public Double getProcessorLoadAverage() {
-        return processorLoadAverage;
-    }
-
-    public void setProcessorLoadAverage(Double processorLoadAverage) {
-        this.processorLoadAverage = processorLoadAverage;
-    }
-
-    /**
-     * The total size of the heap.
-     *
-     * @return
-     */
-    public String getTotalHeap() {
-        return totalHeap;
-    }
-
-    public void setTotalHeap(String totalHeap) {
-        this.totalHeap = totalHeap;
-    }
-
-    /**
-     * The total size of non-heap.
-     *
-     * @return
-     */
-    public String getTotalNonHeap() {
-        return totalNonHeap;
-    }
-
-    public void setTotalNonHeap(String totalNonHeap) {
-        this.totalNonHeap = totalNonHeap;
-    }
-
-    /**
-     * The total number of threads.
-     *
-     * @return
-     */
-    public Integer getTotalThreads() {
-        return totalThreads;
-    }
-
-    public void setTotalThreads(Integer totalThreads) {
-        this.totalThreads = totalThreads;
-    }
-
-    /**
-     * The amount of used heap.
-     *
-     * @return
-     */
-    public String getUsedHeap() {
-        return usedHeap;
-    }
-
-    public void setUsedHeap(String usedHeap) {
-        this.usedHeap = usedHeap;
-    }
-
-    /**
-     * The amount of used non-heap.
-     *
-     * @return
-     */
-    public String getUsedNonHeap() {
-        return usedNonHeap;
-    }
-
-    public void setUsedNonHeap(String usedNonHeap) {
-        this.usedNonHeap = usedNonHeap;
-    }
-
-    /**
-     * The heap utilization.
-     *
-     * @return
-     */
-    public String getHeapUtilization() {
-        return heapUtilization;
-    }
-
-    public void setHeapUtilization(String heapUtilization) {
-        this.heapUtilization = heapUtilization;
-    }
-
-    /**
-     * The non-heap utilization.
-     *
-     * @return
-     */
-    public String getNonHeapUtilization() {
-        return nonHeapUtilization;
-    }
-
-    public void setNonHeapUtilization(String nonHeapUsage) {
-        this.nonHeapUtilization = nonHeapUsage;
-    }
-
-    /**
-     * The content repository storage usage.
-     *
-     * @return
-     */
-    public Set<StorageUsageDTO> getContentRepositoryStorageUsage() {
-        return contentRepositoryStorageUsage;
-    }
-
-    public void setContentRepositoryStorageUsage(Set<StorageUsageDTO> contentRepositoryStorageUsage) {
-        this.contentRepositoryStorageUsage = contentRepositoryStorageUsage;
-    }
-
-    /**
-     * The flowfile repository storage usage.
-     *
-     * @return
-     */
-    public StorageUsageDTO getFlowFileRepositoryStorageUsage() {
-        return flowFileRepositoryStorageUsage;
-    }
-
-    public void setFlowFileRepositoryStorageUsage(StorageUsageDTO flowFileRepositoryStorageUsage) {
-        this.flowFileRepositoryStorageUsage = flowFileRepositoryStorageUsage;
-    }
-
-    /**
-     * Garbage collection details.
-     *
-     * @return
-     */
-    public Set<GarbageCollectionDTO> getGarbageCollection() {
-        return garbageCollection;
-    }
-
-    public void setGarbageCollection(Set<GarbageCollectionDTO> garbageCollection) {
-        this.garbageCollection = garbageCollection;
-    }
-
-    /**
-     * When these diagnostics were generated.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    public Date getStatsLastRefreshed() {
-        return statsLastRefreshed;
-    }
-
-    public void setStatsLastRefreshed(Date statsLastRefreshed) {
-        this.statsLastRefreshed = statsLastRefreshed;
-    }
-
-    /**
-     * Details of storage usage.
-     */
-    @XmlType(name = "storageUsage")
-    public static class StorageUsageDTO {
-
-        private String identifier;
-        private String freeSpace;
-        private String totalSpace;
-        private String usedSpace;
-        private Long freeSpaceBytes;
-        private Long totalSpaceBytes;
-        private Long usedSpaceBytes;
-        private String utilization;
-
-        /**
-         * The identifier for this storage location.
-         *
-         * @return
-         */
-        public String getIdentifier() {
-            return identifier;
-        }
-
-        public void setIdentifier(String identifier) {
-            this.identifier = identifier;
-        }
-
-        /**
-         * The amount of free space.
-         *
-         * @return
-         */
-        public String getFreeSpace() {
-            return freeSpace;
-        }
-
-        public void setFreeSpace(String freeSpace) {
-            this.freeSpace = freeSpace;
-        }
-
-        /**
-         * The amount of total space.
-         *
-         * @param freeSpace
-         */
-        public String getTotalSpace() {
-            return totalSpace;
-        }
-
-        public void setTotalSpace(String totalSpace) {
-            this.totalSpace = totalSpace;
-        }
-
-        /**
-         * The amount of used space.
-         *
-         * @return
-         */
-        public String getUsedSpace() {
-            return usedSpace;
-        }
-
-        public void setUsedSpace(String usedSpace) {
-            this.usedSpace = usedSpace;
-        }
-
-        /**
-         * The utilization of this storage location.
-         *
-         * @return
-         */
-        public String getUtilization() {
-            return utilization;
-        }
-
-        public void setUtilization(String utilization) {
-            this.utilization = utilization;
-        }
-
-        /**
-         * The number of bytes of free space.
-         *
-         * @return
-         */
-        public Long getFreeSpaceBytes() {
-            return freeSpaceBytes;
-        }
-
-        public void setFreeSpaceBytes(Long freeSpaceBytes) {
-            this.freeSpaceBytes = freeSpaceBytes;
-        }
-
-        /**
-         * The number of bytes of total space.
-         *
-         * @return
-         */
-        public Long getTotalSpaceBytes() {
-            return totalSpaceBytes;
-        }
-
-        public void setTotalSpaceBytes(Long totalSpaceBytes) {
-            this.totalSpaceBytes = totalSpaceBytes;
-        }
-
-        /**
-         * The number of bytes of used space.
-         *
-         * @return
-         */
-        public Long getUsedSpaceBytes() {
-            return usedSpaceBytes;
-        }
-
-        public void setUsedSpaceBytes(Long usedSpaceBytes) {
-            this.usedSpaceBytes = usedSpaceBytes;
-        }
-    }
-
-    /**
-     * Details for garbage collection.
-     */
-    @XmlType(name = "garbageCollection")
-    public static class GarbageCollectionDTO {
-
-        private String name;
-        private long collectionCount;
-        private String collectionTime;
-
-        /**
-         * The name of the garbage collector.
-         *
-         * @return
-         */
-        public String getName() {
-            return name;
-        }
-
-        public void setName(String name) {
-            this.name = name;
-        }
-
-        public long getCollectionCount() {
-            return collectionCount;
-        }
-
-        /**
-         * The number of times garbage collection has run.
-         *
-         * @param collectionCount
-         */
-        public void setCollectionCount(long collectionCount) {
-            this.collectionCount = collectionCount;
-        }
-
-        /**
-         * The total amount of time spent garbage collecting.
-         *
-         * @return
-         */
-        public String getCollectionTime() {
-            return collectionTime;
-        }
-
-        public void setCollectionTime(String collectionTime) {
-            this.collectionTime = collectionTime;
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/TemplateDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/TemplateDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/TemplateDTO.java
deleted file mode 100644
index 156a6e9..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/TemplateDTO.java
+++ /dev/null
@@ -1,117 +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.api.dto;
-
-import java.util.Date;
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.DateTimeAdapter;
-
-/**
- * Defines a template.
- */
-@XmlRootElement(name = "template")
-public class TemplateDTO {
-
-    private String uri;
-
-    private String id;
-    private String name;
-    private String description;
-    private Date timestamp;
-
-    private FlowSnippetDTO snippet;
-
-    /**
-     * The id for this template.
-     *
-     * @return
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The uri for this template.
-     *
-     * @return
-     */
-    public String getUri() {
-        return uri;
-    }
-
-    public void setUri(String uri) {
-        this.uri = uri;
-    }
-
-    /**
-     * The name of this template.
-     *
-     * @return
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * The description of this template.
-     *
-     * @return
-     */
-    public String getDescription() {
-        return description;
-    }
-
-    public void setDescription(String description) {
-        this.description = description;
-    }
-
-    /**
-     * The timestamp when this template was created.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(DateTimeAdapter.class)
-    public Date getTimestamp() {
-        return timestamp;
-    }
-
-    public void setTimestamp(Date timestamp) {
-        this.timestamp = timestamp;
-    }
-
-    /**
-     * The snippet in this template.
-     *
-     * @return
-     */
-    public FlowSnippetDTO getSnippet() {
-        return snippet;
-    }
-
-    public void setSnippet(FlowSnippetDTO snippet) {
-        this.snippet = snippet;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/UserDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/UserDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/UserDTO.java
deleted file mode 100644
index 85c82dc..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/UserDTO.java
+++ /dev/null
@@ -1,177 +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.api.dto;
-
-import java.util.Date;
-import java.util.Set;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.DateTimeAdapter;
-
-/**
- * A user of this NiFi.
- */
-@XmlType(name = "user")
-public class UserDTO {
-
-    private String id;
-    private String dn;
-    private String userName;
-    private String userGroup;
-    private String justification;
-    private Date creation;
-    private String status;
-
-    private Date lastVerified;
-    private Date lastAccessed;
-    private Set<String> authorities;
-
-    /**
-     * The user id.
-     *
-     * @return
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The users authorities.
-     *
-     * @return
-     */
-    public Set<String> getAuthorities() {
-        return authorities;
-    }
-
-    public void setAuthorities(Set<String> authorities) {
-        this.authorities = authorities;
-    }
-
-    /**
-     * The creation time for this users account.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(DateTimeAdapter.class)
-    public Date getCreation() {
-        return creation;
-    }
-
-    public void setCreation(Date creation) {
-        this.creation = creation;
-    }
-
-    /**
-     * The users DN.
-     *
-     * @return
-     */
-    public String getDn() {
-        return dn;
-    }
-
-    public void setDn(String dn) {
-        this.dn = dn;
-    }
-
-    /**
-     * The users name. If the name could not be extracted from the DN, this
-     * value will be the entire DN.
-     *
-     * @return
-     */
-    public String getUserName() {
-        return userName;
-    }
-
-    public void setUserName(String userName) {
-        this.userName = userName;
-    }
-
-    /**
-     * The user group.
-     *
-     * @return
-     */
-    public String getUserGroup() {
-        return userGroup;
-    }
-
-    public void setUserGroup(String userGroup) {
-        this.userGroup = userGroup;
-    }
-
-    /**
-     * The users account justification.
-     *
-     * @return
-     */
-    public String getJustification() {
-        return justification;
-    }
-
-    public void setJustification(String justification) {
-        this.justification = justification;
-    }
-
-    /**
-     * The time that the user last accessed the system.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(DateTimeAdapter.class)
-    public Date getLastAccessed() {
-        return lastAccessed;
-    }
-
-    public void setLastAccessed(Date lastAccessed) {
-        this.lastAccessed = lastAccessed;
-    }
-
-    /**
-     * The time that the users credentials were last verified.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(DateTimeAdapter.class)
-    public Date getLastVerified() {
-        return lastVerified;
-    }
-
-    public void setLastVerified(Date lastVerified) {
-        this.lastVerified = lastVerified;
-    }
-
-    /**
-     * The status of the users account.
-     *
-     * @return
-     */
-    public String getStatus() {
-        return status;
-    }
-
-    public void setStatus(String status) {
-        this.status = status;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/UserGroupDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/UserGroupDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/UserGroupDTO.java
deleted file mode 100644
index 285c355..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/UserGroupDTO.java
+++ /dev/null
@@ -1,84 +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.api.dto;
-
-import java.util.Set;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * A user group in this NiFi.
- */
-@XmlType(name = "userGroup")
-public class UserGroupDTO {
-
-    private String group;
-    private Set<String> userIds;
-    private Set<String> authorities;
-    private String status;
-
-    /**
-     * The user group.
-     *
-     * @return
-     */
-    public String getGroup() {
-        return group;
-    }
-
-    public void setGroup(String group) {
-        this.group = group;
-    }
-
-    /**
-     * The users in this group.
-     *
-     * @return
-     */
-    public Set<String> getUserIds() {
-        return userIds;
-    }
-
-    public void setUserIds(Set<String> userIds) {
-        this.userIds = userIds;
-    }
-
-    /**
-     * The status of the users account.
-     *
-     * @return
-     */
-    public String getStatus() {
-        return status;
-    }
-
-    public void setStatus(String status) {
-        this.status = status;
-    }
-
-    /**
-     * The users authorities.
-     *
-     * @return
-     */
-    public Set<String> getAuthorities() {
-        return authorities;
-    }
-
-    public void setAuthorities(Set<String> authorities) {
-        this.authorities = authorities;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/ActionDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/ActionDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/ActionDTO.java
deleted file mode 100644
index 8402480..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/ActionDTO.java
+++ /dev/null
@@ -1,176 +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.api.dto.action;
-
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.action.component.details.ComponentDetailsDTO;
-import org.apache.nifi.web.api.dto.action.details.ActionDetailsDTO;
-import org.apache.nifi.web.api.dto.util.DateTimeAdapter;
-
-/**
- * An action performed in this NiFi.
- */
-@XmlType(name = "action")
-public class ActionDTO {
-
-    private Integer id;
-    private String userDn;
-    private String userName;
-    private Date timestamp;
-
-    private String sourceId;
-    private String sourceName;
-    private String sourceType;
-    private ComponentDetailsDTO componentDetails;
-
-    private String operation;
-    private ActionDetailsDTO actionDetails;
-
-    /**
-     * The action id.
-     *
-     * @return
-     */
-    public Integer getId() {
-        return id;
-    }
-
-    public void setId(Integer id) {
-        this.id = id;
-    }
-
-    /**
-     * The user dn who perform this action.
-     *
-     * @return
-     */
-    public String getUserDn() {
-        return userDn;
-    }
-
-    public void setUserDn(String userDn) {
-        this.userDn = userDn;
-    }
-
-    /**
-     * The user name who perform this action.
-     *
-     * @return
-     */
-    public String getUserName() {
-        return userName;
-    }
-
-    public void setUserName(String userName) {
-        this.userName = userName;
-    }
-
-    /**
-     * This action's timestamp.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(DateTimeAdapter.class)
-    public Date getTimestamp() {
-        return timestamp;
-    }
-
-    public void setTimestamp(Date timestamp) {
-        this.timestamp = timestamp;
-    }
-
-    /**
-     * The id of the source component of this action.
-     *
-     * @return
-     */
-    public String getSourceId() {
-        return sourceId;
-    }
-
-    public void setSourceId(String sourceId) {
-        this.sourceId = sourceId;
-    }
-
-    /**
-     * The name of the source component of this action.
-     *
-     * @return
-     */
-    public String getSourceName() {
-        return sourceName;
-    }
-
-    public void setSourceName(String sourceName) {
-        this.sourceName = sourceName;
-    }
-
-    /**
-     * The type of the source component of this action.
-     *
-     * @return
-     */
-    public String getSourceType() {
-        return sourceType;
-    }
-
-    public void setSourceType(String sourceType) {
-        this.sourceType = sourceType;
-    }
-
-    /**
-     * The component details (if any) for this action.
-     *
-     * @return
-     */
-    public ComponentDetailsDTO getComponentDetails() {
-        return componentDetails;
-    }
-
-    public void setComponentDetails(ComponentDetailsDTO componentDetails) {
-        this.componentDetails = componentDetails;
-    }
-
-    /**
-     * The operation being performed in this action.
-     *
-     * @return
-     */
-    public String getOperation() {
-        return operation;
-    }
-
-    public void setOperation(String operation) {
-        this.operation = operation;
-    }
-
-    /**
-     * The action details (if any) for this action.
-     *
-     * @return
-     */
-    public ActionDetailsDTO getActionDetails() {
-        return actionDetails;
-    }
-
-    public void setActionDetails(ActionDetailsDTO actionDetails) {
-        this.actionDetails = actionDetails;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/HistoryDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/HistoryDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/HistoryDTO.java
deleted file mode 100644
index 9ab47ed..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/HistoryDTO.java
+++ /dev/null
@@ -1,74 +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.api.dto.action;
-
-import java.util.Collection;
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * NiFi action history.
- */
-@XmlType(name = "history")
-public class HistoryDTO {
-
-    private Integer total;
-    private Date lastRefreshed;
-    private Collection<ActionDTO> actions;
-
-    /**
-     * The total number of actions.
-     *
-     * @return
-     */
-    public Integer getTotal() {
-        return total;
-    }
-
-    public void setTotal(Integer total) {
-        this.total = total;
-    }
-
-    /**
-     * Timestamp when these records were returned.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    public Date getLastRefreshed() {
-        return lastRefreshed;
-    }
-
-    public void setLastRefreshed(Date lastRefreshed) {
-        this.lastRefreshed = lastRefreshed;
-    }
-
-    /**
-     * The actions for this range.
-     *
-     * @return
-     */
-    public Collection<ActionDTO> getActions() {
-        return actions;
-    }
-
-    public void setActions(Collection<ActionDTO> actions) {
-        this.actions = actions;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/HistoryQueryDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/HistoryQueryDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/HistoryQueryDTO.java
deleted file mode 100644
index e8e11e5..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/HistoryQueryDTO.java
+++ /dev/null
@@ -1,144 +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.api.dto.action;
-
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.DateTimeAdapter;
-
-/**
- * A history query to find desired actions.
- */
-@XmlType(name = "historyQuery")
-public class HistoryQueryDTO {
-
-    private String userName;
-    private String sourceId;
-    private Date startDate;
-    private Date endDate;
-    private Integer offset;
-    private Integer count;
-    private String sortColumn;
-    private String sortOrder;
-
-    /**
-     * The user name.
-     *
-     * @return
-     */
-    public String getUserName() {
-        return userName;
-    }
-
-    public void setUserName(String userName) {
-        this.userName = userName;
-    }
-
-    /**
-     * The source component id.
-     *
-     * @return
-     */
-    public String getSourceId() {
-        return sourceId;
-    }
-
-    public void setSourceId(String sourceId) {
-        this.sourceId = sourceId;
-    }
-
-    /**
-     * The start date.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(DateTimeAdapter.class)
-    public Date getStartDate() {
-        return startDate;
-    }
-
-    public void setStartDate(Date startDate) {
-        this.startDate = startDate;
-    }
-
-    /**
-     * The end date.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(DateTimeAdapter.class)
-    public Date getEndDate() {
-        return endDate;
-    }
-
-    public void setEndDate(Date endDate) {
-        this.endDate = endDate;
-    }
-
-    /**
-     * The offset.
-     *
-     * @return
-     */
-    public Integer getOffset() {
-        return offset;
-    }
-
-    public void setOffset(Integer offset) {
-        this.offset = offset;
-    }
-
-    /**
-     * The desired row count.
-     *
-     * @return
-     */
-    public Integer getCount() {
-        return count;
-    }
-
-    public void setCount(Integer count) {
-        this.count = count;
-    }
-
-    /**
-     * The desired sort column.
-     *
-     * @return
-     */
-    public String getSortColumn() {
-        return sortColumn;
-    }
-
-    public void setSortColumn(String sortColumn) {
-        this.sortColumn = sortColumn;
-    }
-
-    /**
-     * The desired sort order.
-     *
-     * @return
-     */
-    public String getSortOrder() {
-        return sortOrder;
-    }
-
-    public void setSortOrder(String sortOrder) {
-        this.sortOrder = sortOrder;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.java
deleted file mode 100644
index 58086ce..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.java
+++ /dev/null
@@ -1,32 +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.api.dto.action.component.details;
-
-import javax.xml.bind.annotation.XmlSeeAlso;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- *
- */
-@XmlType(name = "componentDetails")
-@XmlSeeAlso({
-    ProcessorDetailsDTO.class,
-    RemoteProcessGroupDetailsDTO.class
-})
-public class ComponentDetailsDTO {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.java
deleted file mode 100644
index 3523f62..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.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.web.api.dto.action.component.details;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Processor details for an action.
- */
-@XmlType(name = "processorDetails")
-public class ProcessorDetailsDTO extends ComponentDetailsDTO {
-
-    private String type;
-
-    /**
-     * The processors type.
-     *
-     * @return
-     */
-    public String getType() {
-        return type;
-    }
-
-    public void setType(String type) {
-        this.type = type;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/RemoteProcessGroupDetailsDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/RemoteProcessGroupDetailsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/RemoteProcessGroupDetailsDTO.java
deleted file mode 100644
index 2809398..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/RemoteProcessGroupDetailsDTO.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.web.api.dto.action.component.details;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Remote process group details for an action.
- */
-@XmlType(name = "remoteProcessGroupDetails")
-public class RemoteProcessGroupDetailsDTO extends ComponentDetailsDTO {
-
-    private String uri;
-
-    /**
-     * The URI of the remote process group.
-     *
-     * @return
-     */
-    public String getUri() {
-        return uri;
-    }
-
-    public void setUri(String uri) {
-        this.uri = uri;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ActionDetailsDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ActionDetailsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ActionDetailsDTO.java
deleted file mode 100644
index 4074cba..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ActionDetailsDTO.java
+++ /dev/null
@@ -1,34 +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.api.dto.action.details;
-
-import javax.xml.bind.annotation.XmlSeeAlso;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- *
- */
-@XmlType(name = "details")
-@XmlSeeAlso({
-    ConfigureDetailsDTO.class,
-    MoveDetailsDTO.class,
-    ConnectDetailsDTO.class,
-    PurgeDetailsDTO.class
-})
-public class ActionDetailsDTO {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ConfigureDetailsDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ConfigureDetailsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ConfigureDetailsDTO.java
deleted file mode 100644
index 4011b00..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ConfigureDetailsDTO.java
+++ /dev/null
@@ -1,69 +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.api.dto.action.details;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Configuration details for an Action.
- */
-@XmlType(name = "configureDetails")
-public class ConfigureDetailsDTO extends ActionDetailsDTO {
-
-    private String name;
-    private String previousValue;
-    private String value;
-
-    /**
-     * The name of the property that was modified.
-     *
-     * @return
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * The previous value.
-     *
-     * @return
-     */
-    public String getPreviousValue() {
-        return previousValue;
-    }
-
-    public void setPreviousValue(String previousValue) {
-        this.previousValue = previousValue;
-    }
-
-    /**
-     * The new value.
-     *
-     * @return
-     */
-    public String getValue() {
-        return value;
-    }
-
-    public void setValue(String value) {
-        this.value = value;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ConnectDetailsDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ConnectDetailsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ConnectDetailsDTO.java
deleted file mode 100644
index ba88bd1..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ConnectDetailsDTO.java
+++ /dev/null
@@ -1,126 +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.api.dto.action.details;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Details for connect Actions.
- */
-@XmlType(name = "connectDetails")
-public class ConnectDetailsDTO extends ActionDetailsDTO {
-
-    private String sourceId;
-    private String sourceName;
-    private String sourceType;
-    private String relationship;
-    private String destinationId;
-    private String destinationName;
-    private String destinationType;
-
-    /**
-     * The id of the source of the connection.
-     *
-     * @return
-     */
-    public String getSourceId() {
-        return sourceId;
-    }
-
-    public void setSourceId(String sourceId) {
-        this.sourceId = sourceId;
-    }
-
-    /**
-     * The name of the source of the connection.
-     *
-     * @return
-     */
-    public String getSourceName() {
-        return sourceName;
-    }
-
-    public void setSourceName(String sourceName) {
-        this.sourceName = sourceName;
-    }
-
-    /**
-     * The type of the source of the connection.
-     *
-     * @return
-     */
-    public String getSourceType() {
-        return sourceType;
-    }
-
-    public void setSourceType(String sourceType) {
-        this.sourceType = sourceType;
-    }
-
-    /**
-     * The name of the relationship that was connected.
-     *
-     * @return
-     */
-    public String getRelationship() {
-        return relationship;
-    }
-
-    public void setRelationship(String relationship) {
-        this.relationship = relationship;
-    }
-
-    /**
-     * The id of the destination of the connection.
-     *
-     * @return
-     */
-    public String getDestinationId() {
-        return destinationId;
-    }
-
-    public void setDestinationId(String destinationId) {
-        this.destinationId = destinationId;
-    }
-
-    /**
-     * The name of the destination of the connection.
-     *
-     * @return
-     */
-    public String getDestinationName() {
-        return destinationName;
-    }
-
-    public void setDestinationName(String destinationName) {
-        this.destinationName = destinationName;
-    }
-
-    /**
-     * The type of the destination of the connection.
-     *
-     * @return
-     */
-    public String getDestinationType() {
-        return destinationType;
-    }
-
-    public void setDestinationType(String destinationType) {
-        this.destinationType = destinationType;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/MoveDetailsDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/MoveDetailsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/MoveDetailsDTO.java
deleted file mode 100644
index c6f0450..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/MoveDetailsDTO.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.web.api.dto.action.details;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Details of the move operation.
- */
-@XmlType(name = "moveDetails")
-public class MoveDetailsDTO extends ActionDetailsDTO {
-
-    private String previousGroupId;
-    private String previousGroup;
-    private String groupId;
-    private String group;
-
-    /**
-     * The id of the group the components previously belonged to.
-     *
-     * @return
-     */
-    public String getPreviousGroupId() {
-        return previousGroupId;
-    }
-
-    public void setPreviousGroupId(String previousGroupId) {
-        this.previousGroupId = previousGroupId;
-    }
-
-    /**
-     * The name of the group of the components previously belonged to.
-     *
-     * @return
-     */
-    public String getPreviousGroup() {
-        return previousGroup;
-    }
-
-    public void setPreviousGroup(String previousGroup) {
-        this.previousGroup = previousGroup;
-    }
-
-    /**
-     * The id of the group the components belong to.
-     *
-     * @return
-     */
-    public String getGroupId() {
-        return groupId;
-    }
-
-    public void setGroupId(String groupId) {
-        this.groupId = groupId;
-    }
-
-    /**
-     * The name of the group the components belong to.
-     *
-     * @return
-     */
-    public String getGroup() {
-        return group;
-    }
-
-    public void setGroup(String group) {
-        this.group = group;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/PurgeDetailsDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/PurgeDetailsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/PurgeDetailsDTO.java
deleted file mode 100644
index b5a5bcb..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/PurgeDetailsDTO.java
+++ /dev/null
@@ -1,45 +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.api.dto.action.details;
-
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.DateTimeAdapter;
-
-/**
- * Details of a purge operation.
- */
-@XmlType(name = "purgeDetails")
-public class PurgeDetailsDTO extends ActionDetailsDTO {
-
-    private Date endDate;
-
-    /**
-     * The end date for this purge action.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(DateTimeAdapter.class)
-    public Date getEndDate() {
-        return endDate;
-    }
-
-    public void setEndDate(Date endDate) {
-        this.endDate = endDate;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/AttributeDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/AttributeDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/AttributeDTO.java
deleted file mode 100644
index ed58077..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/AttributeDTO.java
+++ /dev/null
@@ -1,69 +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.api.dto.provenance;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Represents a processor's attribute for a provenance event.
- */
-@XmlType(name = "attribute")
-public class AttributeDTO {
-
-    private String name;
-    private String value;
-    private String previousValue;
-
-    /**
-     * The attribute name.
-     *
-     * @return
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * The attribute value.
-     *
-     * @return
-     */
-    public String getValue() {
-        return value;
-    }
-
-    public void setValue(String value) {
-        this.value = value;
-    }
-
-    /**
-     * The value of this attribute before the event took place.
-     *
-     * @return
-     */
-    public String getPreviousValue() {
-        return previousValue;
-    }
-
-    public void setPreviousValue(String previousValue) {
-        this.previousValue = previousValue;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceDTO.java
deleted file mode 100644
index 90dd119..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceDTO.java
+++ /dev/null
@@ -1,165 +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.api.dto.provenance;
-
-import java.util.Date;
-
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-
-import org.apache.nifi.web.api.dto.util.TimestampAdapter;
-
-/**
- * A provenance submission. Incorporates the request, its current status, and
- * the results.
- */
-@XmlType(name = "provenance")
-public class ProvenanceDTO {
-
-    private String id;
-    private String uri;
-    private String clusterNodeId;
-
-    private Date submissionTime;
-    private Date expiration;
-
-    private Integer percentCompleted;
-    private Boolean finished;
-
-    private ProvenanceRequestDTO request;
-    private ProvenanceResultsDTO results;
-
-    /**
-     * The id of this provenance query.
-     *
-     * @return
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The URI for this query. Used for obtaining the requests at a later time.
-     *
-     * @return
-     */
-    public String getUri() {
-        return uri;
-    }
-
-    public void setUri(String uri) {
-        this.uri = uri;
-    }
-
-    /**
-     * The id of the node in the cluster where this provenance originated.
-     *
-     * @return
-     */
-    public String getClusterNodeId() {
-        return clusterNodeId;
-    }
-
-    public void setClusterNodeId(String clusterNodeId) {
-        this.clusterNodeId = clusterNodeId;
-    }
-
-    /**
-     * The time the query was submitted.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimestampAdapter.class)
-    public Date getSubmissionTime() {
-        return submissionTime;
-    }
-
-    public void setSubmissionTime(Date submissionTime) {
-        this.submissionTime = submissionTime;
-    }
-
-    /**
-     * The expiration time of the query results.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimestampAdapter.class)
-    public Date getExpiration() {
-        return expiration;
-    }
-
-    public void setExpiration(Date expiration) {
-        this.expiration = expiration;
-    }
-
-    /**
-     * The percent completed.
-     *
-     * @return
-     */
-    public Integer getPercentCompleted() {
-        return percentCompleted;
-    }
-
-    public void setPercentCompleted(Integer percentCompleted) {
-        this.percentCompleted = percentCompleted;
-    }
-
-    /**
-     * Whether the query has finished.
-     *
-     * @return
-     */
-    public Boolean isFinished() {
-        return finished;
-    }
-
-    public void setFinished(Boolean finished) {
-        this.finished = finished;
-    }
-
-    /**
-     * The provenance request.
-     *
-     * @return
-     */
-    public ProvenanceRequestDTO getRequest() {
-        return request;
-    }
-
-    public void setRequest(ProvenanceRequestDTO request) {
-        this.request = request;
-    }
-
-    /**
-     * The results of this query.
-     *
-     * @return
-     */
-    public ProvenanceResultsDTO getResults() {
-        return results;
-    }
-
-    public void setResults(ProvenanceResultsDTO results) {
-        this.results = results;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceEventDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceEventDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceEventDTO.java
deleted file mode 100644
index b1ead42..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceEventDTO.java
+++ /dev/null
@@ -1,630 +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.api.dto.provenance;
-
-import java.util.Collection;
-import java.util.Date;
-import java.util.List;
-
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-
-import org.apache.nifi.web.api.dto.util.TimestampAdapter;
-
-/**
- * A provenance event.
- */
-@XmlType(name = "provenanceEvent")
-public class ProvenanceEventDTO {
-
-    private String id;
-
-    // in search results table
-    private Long eventId;
-    private Date eventTime;
-    private Long eventDuration;
-    private Long lineageDuration;
-    private String eventType;
-    private String flowFileUuid;
-    private String fileSize;
-    private Long fileSizeBytes;
-    private String clusterNodeId;    // include when clustered
-    private String clusterNodeAddress; // include when clustered
-
-    private String groupId;
-    private String componentId;
-    private String componentType;
-    private String componentName;
-    private String sourceSystemFlowFileId;
-    private String alternateIdentifierUri;
-    private Collection<AttributeDTO> attributes;
-    private List<String> parentUuids;
-    private List<String> childUuids;
-
-    private String transitUri;
-
-    private String relationship;
-    private String details;
-
-    // content
-    private Boolean contentEqual;
-    private Boolean inputContentAvailable;
-    private String inputContentClaimSection;
-    private String inputContentClaimContainer;
-    private String inputContentClaimIdentifier;
-    private Long inputContentClaimOffset;
-    private String inputContentClaimFileSize;
-    private Long inputContentClaimFileSizeBytes;
-    private Boolean outputContentAvailable;
-    private String outputContentClaimSection;
-    private String outputContentClaimContainer;
-    private String outputContentClaimIdentifier;
-    private Long outputContentClaimOffset;
-    private String outputContentClaimFileSize;
-    private Long outputContentClaimFileSizeBytes;
-
-    // replay
-    private Boolean replayAvailable;
-    private String replayExplanation;
-    private String sourceConnectionIdentifier;
-
-    /**
-     * The event uuid.
-     *
-     * @return
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The event id.
-     *
-     * @return
-     */
-    public Long getEventId() {
-        return eventId;
-    }
-
-    public void setEventId(Long eventId) {
-        this.eventId = eventId;
-    }
-
-    /**
-     * The time the event occurred.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimestampAdapter.class)
-    public Date getEventTime() {
-        return eventTime;
-    }
-
-    public void setEventTime(Date eventTime) {
-        this.eventTime = eventTime;
-    }
-
-    /**
-     * The UUID of the FlowFile for this event.
-     *
-     * @return
-     */
-    public String getFlowFileUuid() {
-        return flowFileUuid;
-    }
-
-    public void setFlowFileUuid(String flowFileUuid) {
-        this.flowFileUuid = flowFileUuid;
-    }
-
-    /**
-     * The size of the FlowFile for this event.
-     *
-     * @return
-     */
-    public String getFileSize() {
-        return fileSize;
-    }
-
-    public void setFileSize(String fileSize) {
-        this.fileSize = fileSize;
-    }
-
-    /**
-     * The size of the FlowFile in bytes for this event.
-     *
-     * @return
-     */
-    public Long getFileSizeBytes() {
-        return fileSizeBytes;
-    }
-
-    public void setFileSizeBytes(Long fileSizeBytes) {
-        this.fileSizeBytes = fileSizeBytes;
-    }
-
-    /**
-     * The type of this event.
-     *
-     * @return
-     */
-    public String getEventType() {
-        return eventType;
-    }
-
-    public void setEventType(String eventType) {
-        this.eventType = eventType;
-    }
-
-    /**
-     * The attributes for the FlowFile for this event.
-     *
-     * @return
-     */
-    public Collection<AttributeDTO> getAttributes() {
-        return attributes;
-    }
-
-    public void setAttributes(Collection<AttributeDTO> attributes) {
-        this.attributes = attributes;
-    }
-
-    /**
-     * The id of the group that this component resides in. If the component is
-     * no longer in the flow, the group id will not be set.
-     *
-     * @return
-     */
-    public String getGroupId() {
-        return groupId;
-    }
-
-    public void setGroupId(String groupId) {
-        this.groupId = groupId;
-    }
-
-    /**
-     * The id of the component that generated this event.
-     *
-     * @return
-     */
-    public String getComponentId() {
-        return componentId;
-    }
-
-    public void setComponentId(String componentId) {
-        this.componentId = componentId;
-    }
-
-    /**
-     * The name of the component that generated this event.
-     *
-     * @return
-     */
-    public String getComponentName() {
-        return componentName;
-    }
-
-    public void setComponentName(String componentName) {
-        this.componentName = componentName;
-    }
-
-    /**
-     * The type of the component that generated this event.
-     *
-     * @return
-     */
-    public String getComponentType() {
-        return componentType;
-    }
-
-    public void setComponentType(String componentType) {
-        this.componentType = componentType;
-    }
-
-    /**
-     * The source/destination system URI if the event was a RECEIVE/SEND.
-     *
-     * @return
-     */
-    public String getTransitUri() {
-        return transitUri;
-    }
-
-    public void setTransitUri(String transitUri) {
-        this.transitUri = transitUri;
-    }
-
-    /**
-     * The alternate identifier URI for the FlowFile for this event.
-     *
-     * @return
-     */
-    public String getAlternateIdentifierUri() {
-        return alternateIdentifierUri;
-    }
-
-    public void setAlternateIdentifierUri(String alternateIdentifierUri) {
-        this.alternateIdentifierUri = alternateIdentifierUri;
-    }
-
-    /**
-     * The identifier of the node where this event originated.
-     *
-     * @return
-     */
-    public String getClusterNodeId() {
-        return clusterNodeId;
-    }
-
-    public void setClusterNodeId(String clusterNodeId) {
-        this.clusterNodeId = clusterNodeId;
-    }
-
-    /**
-     * The label to use to show which node this event originated from.
-     *
-     * @return
-     */
-    public String getClusterNodeAddress() {
-        return clusterNodeAddress;
-    }
-
-    public void setClusterNodeAddress(String clusterNodeAddress) {
-        this.clusterNodeAddress = clusterNodeAddress;
-    }
-
-    /**
-     * The parent uuids for this event.
-     *
-     * @return
-     */
-    public List<String> getParentUuids() {
-        return parentUuids;
-    }
-
-    public void setParentUuids(List<String> parentUuids) {
-        this.parentUuids = parentUuids;
-    }
-
-    /**
-     * The child uuids for this event.
-     *
-     * @return
-     */
-    public List<String> getChildUuids() {
-        return childUuids;
-    }
-
-    public void setChildUuids(List<String> childUuids) {
-        this.childUuids = childUuids;
-    }
-
-    /**
-     * The duration of the event, in milliseconds.
-     *
-     * @return
-     */
-    public Long getEventDuration() {
-        return eventDuration;
-    }
-
-    public void setEventDuration(Long eventDuration) {
-        this.eventDuration = eventDuration;
-    }
-
-    /**
-     * The duration since the lineage began, in milliseconds.
-     *
-     * @return
-     */
-    public Long getLineageDuration() {
-        return lineageDuration;
-    }
-
-    public void setLineageDuration(Long lineageDuration) {
-        this.lineageDuration = lineageDuration;
-    }
-
-    /**
-     * The source system FlowFile id.
-     *
-     * @return
-     */
-    public String getSourceSystemFlowFileId() {
-        return sourceSystemFlowFileId;
-    }
-
-    public void setSourceSystemFlowFileId(String sourceSystemFlowFileId) {
-        this.sourceSystemFlowFileId = sourceSystemFlowFileId;
-    }
-
-    /**
-     * If this represents a route event, this is the relationship to which the
-     * flowfile was routed.
-     *
-     * @return
-     */
-    public String getRelationship() {
-        return relationship;
-    }
-
-    public void setRelationship(String relationship) {
-        this.relationship = relationship;
-    }
-
-    /**
-     * The event details.
-     *
-     * @return
-     */
-    public String getDetails() {
-        return details;
-    }
-
-    public void setDetails(String details) {
-        this.details = details;
-    }
-
-    /**
-     * Whether or not the input and output content claim is the same.
-     *
-     * @return
-     */
-    public Boolean getContentEqual() {
-        return contentEqual;
-    }
-
-    public void setContentEqual(Boolean contentEqual) {
-        this.contentEqual = contentEqual;
-    }
-
-    /**
-     * Returns whether or not the output content is still available.
-     *
-     * @return
-     */
-    public Boolean getOutputContentAvailable() {
-        return outputContentAvailable;
-    }
-
-    public void setOutputContentAvailable(Boolean outputContentAvailable) {
-        this.outputContentAvailable = outputContentAvailable;
-    }
-
-    /**
-     * Returns the Section in which the output Content Claim lives, or
-     * <code>null</code> if no Content Claim exists.
-     *
-     * @return
-     */
-    public String getOutputContentClaimSection() {
-        return outputContentClaimSection;
-    }
-
-    public void setOutputContentClaimSection(String contentClaimSection) {
-        this.outputContentClaimSection = contentClaimSection;
-    }
-
-    /**
-     * Returns the Container in which the output Content Claim lives, or
-     * <code>null</code> if no Content Claim exists.
-     *
-     * @return
-     */
-    public String getOutputContentClaimContainer() {
-        return outputContentClaimContainer;
-    }
-
-    public void setOutputContentClaimContainer(String outputContentClaimContainer) {
-        this.outputContentClaimContainer = outputContentClaimContainer;
-    }
-
-    /**
-     * Returns the Identifier of the output Content Claim, or <code>null</code>
-     * if no Content Claim exists.
-     *
-     * @return
-     */
-    public String getOutputContentClaimIdentifier() {
-        return outputContentClaimIdentifier;
-    }
-
-    public void setOutputContentClaimIdentifier(String outputContentClaimIdentifier) {
-        this.outputContentClaimIdentifier = outputContentClaimIdentifier;
-    }
-
-    /**
-     * Returns the offset into the the output Content Claim where the FlowFile's
-     * content begins, or <code>null</code> if no Content Claim exists.
-     *
-     * @return
-     */
-    public Long getOutputContentClaimOffset() {
-        return outputContentClaimOffset;
-    }
-
-    public void setOutputContentClaimOffset(Long outputContentClaimOffset) {
-        this.outputContentClaimOffset = outputContentClaimOffset;
-    }
-
-    /**
-     * Returns the formatted file size of the input content claim.
-     *
-     * @return
-     */
-    public String getOutputContentClaimFileSize() {
-        return outputContentClaimFileSize;
-    }
-
-    public void setOutputContentClaimFileSize(String outputContentClaimFileSize) {
-        this.outputContentClaimFileSize = outputContentClaimFileSize;
-    }
-
-    /**
-     * Returns the number of bytes of the input content claim.
-     *
-     * @return
-     */
-    public Long getOutputContentClaimFileSizeBytes() {
-        return outputContentClaimFileSizeBytes;
-    }
-
-    public void setOutputContentClaimFileSizeBytes(Long outputContentClaimFileSizeBytes) {
-        this.outputContentClaimFileSizeBytes = outputContentClaimFileSizeBytes;
-    }
-
-    /**
-     * Returns whether or not the input content is still available.
-     *
-     * @return
-     */
-    public Boolean getInputContentAvailable() {
-        return inputContentAvailable;
-    }
-
-    public void setInputContentAvailable(Boolean inputContentAvailable) {
-        this.inputContentAvailable = inputContentAvailable;
-    }
-
-    /**
-     * Returns the Section in which the input Content Claim lives, or
-     * <code>null</code> if no Content Claim exists.
-     *
-     * @return
-     */
-    public String getInputContentClaimSection() {
-        return inputContentClaimSection;
-    }
-
-    public void setInputContentClaimSection(String inputContentClaimSection) {
-        this.inputContentClaimSection = inputContentClaimSection;
-    }
-
-    /**
-     * Returns the Container in which the input Content Claim lives, or
-     * <code>null</code> if no Content Claim exists.
-     *
-     * @return
-     */
-    public String getInputContentClaimContainer() {
-        return inputContentClaimContainer;
-    }
-
-    public void setInputContentClaimContainer(String inputContentClaimContainer) {
-        this.inputContentClaimContainer = inputContentClaimContainer;
-    }
-
-    /**
-     * Returns the Identifier of the input Content Claim, or <code>null</code>
-     * if no Content Claim exists.
-     *
-     * @return
-     */
-    public String getInputContentClaimIdentifier() {
-        return inputContentClaimIdentifier;
-    }
-
-    public void setInputContentClaimIdentifier(String inputContentClaimIdentifier) {
-        this.inputContentClaimIdentifier = inputContentClaimIdentifier;
-    }
-
-    /**
-     * Returns the offset into the the input Content Claim where the FlowFile's
-     * content begins, or <code>null</code> if no Content Claim exists.
-     *
-     * @return
-     */
-    public Long getInputContentClaimOffset() {
-        return inputContentClaimOffset;
-    }
-
-    public void setInputContentClaimOffset(Long inputContentClaimOffset) {
-        this.inputContentClaimOffset = inputContentClaimOffset;
-    }
-
-    /**
-     * Returns the formatted file size of the input content claim.
-     *
-     * @return
-     */
-    public String getInputContentClaimFileSize() {
-        return inputContentClaimFileSize;
-    }
-
-    public void setInputContentClaimFileSize(String inputContentClaimFileSize) {
-        this.inputContentClaimFileSize = inputContentClaimFileSize;
-    }
-
-    /**
-     * Returns the number of bytes of the input content claim.
-     *
-     * @return
-     */
-    public Long getInputContentClaimFileSizeBytes() {
-        return inputContentClaimFileSizeBytes;
-    }
-
-    public void setInputContentClaimFileSizeBytes(Long inputContentClaimFileSizeBytes) {
-        this.inputContentClaimFileSizeBytes = inputContentClaimFileSizeBytes;
-    }
-
-    /**
-     * Returns whether or not replay is available.
-     *
-     * @return
-     */
-    public Boolean getReplayAvailable() {
-        return replayAvailable;
-    }
-
-    public void setReplayAvailable(Boolean replayAvailable) {
-        this.replayAvailable = replayAvailable;
-    }
-
-    /**
-     * Returns the explanation as to why replay is unavailable.
-     *
-     * @return
-     */
-    public String getReplayExplanation() {
-        return replayExplanation;
-    }
-
-    public void setReplayExplanation(String replayExplanation) {
-        this.replayExplanation = replayExplanation;
-    }
-
-    /**
-     * Returns identifier of the FlowFile Queue / Connection from which the
-     * FlowFile was pulled to generate this event, or <code>null</code> if
-     * either the queue is unknown or the FlowFile was created by this event.
-     *
-     * @return
-     */
-    public String getSourceConnectionIdentifier() {
-        return sourceConnectionIdentifier;
-    }
-
-    public void setSourceConnectionIdentifier(String sourceConnectionIdentifier) {
-        this.sourceConnectionIdentifier = sourceConnectionIdentifier;
-    }
-}


[29/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/UserService.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/UserService.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/UserService.java
deleted file mode 100644
index 86256fd..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/UserService.java
+++ /dev/null
@@ -1,161 +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.admin.service;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.nifi.authorization.Authority;
-import org.apache.nifi.authorization.DownloadAuthorization;
-import org.apache.nifi.user.NiFiUser;
-import org.apache.nifi.user.NiFiUserGroup;
-
-/**
- * Manages NiFi user accounts.
- */
-public interface UserService {
-
-    /**
-     * Creates a new user account using the specified dn and justification.
-     *
-     * @param dn
-     * @param justification
-     * @return
-     */
-    NiFiUser createPendingUserAccount(String dn, String justification);
-
-    /**
-     * Determines if there are any PENDING user accounts present.
-     *
-     * @return
-     */
-    Boolean hasPendingUserAccount();
-
-    /**
-     * Determines if the users in the dnChain are authorized to download content 
-     * with the specified attributes.
-     * 
-     * @param dnChain
-     * @param attributes
-     * @return 
-     */
-    DownloadAuthorization authorizeDownload(List<String> dnChain, Map<String, String> attributes);
-    
-    /**
-     * Updates a user group using the specified group comprised of the specified
-     * users. Returns all the users that are currently in the specified group.
-     *
-     * @param group
-     * @param userIds
-     * @param authorities
-     * @return
-     */
-    NiFiUserGroup updateGroup(String group, Set<String> userIds, Set<Authority> authorities);
-
-    /**
-     * Authorizes the user specified.
-     *
-     * @param dn
-     * @return
-     */
-    NiFiUser checkAuthorization(String dn);
-
-    /**
-     * Deletes the user with the specified id.
-     *
-     * @param id
-     */
-    void deleteUser(String id);
-
-    /**
-     * Disables the specified users account.
-     *
-     * @param id
-     * @return
-     */
-    NiFiUser disable(String id);
-
-    /**
-     * Disables the specified user group.
-     *
-     * @param group
-     * @return
-     */
-    NiFiUserGroup disableGroup(String group);
-
-    /**
-     * Updates the specified user with the specified authorities.
-     *
-     * @param id
-     * @param authorities
-     * @return
-     */
-    NiFiUser update(String id, Set<Authority> authorities);
-
-    /**
-     * Invalidates the specified user account.
-     *
-     * @param id
-     */
-    void invalidateUserAccount(String id);
-
-    /**
-     * Invalidates the user accounts associated with the specified user group.
-     *
-     * @param group
-     */
-    void invalidateUserGroupAccount(String group);
-
-    /**
-     * Ungroups the specified group.
-     *
-     * @param group
-     */
-    void ungroup(String group);
-
-    /**
-     * Ungroups the specified user.
-     *
-     * @param id
-     */
-    void ungroupUser(String id);
-
-    /**
-     * Returns a collection of all NiFiUsers.
-     *
-     * @return
-     */
-    Collection<NiFiUser> getUsers();
-
-    /**
-     * Finds the specified user by id.
-     *
-     * @param id
-     * @return
-     */
-    NiFiUser getUserById(String id);
-
-    /**
-     * Finds the specified user by dn.
-     *
-     * @param dn
-     * @return
-     * @throws AdministrationException
-     */
-    NiFiUser getUserByDn(String dn);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AbstractUserAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AbstractUserAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AbstractUserAction.java
deleted file mode 100644
index 41c97fe..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AbstractUserAction.java
+++ /dev/null
@@ -1,97 +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.admin.service.action;
-
-import java.util.Date;
-import java.util.EnumSet;
-import java.util.Set;
-import org.apache.nifi.authorization.Authority;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUser;
-
-/**
- *
- * @param <T>
- */
-public abstract class AbstractUserAction<T> implements AdministrationAction<T> {
-
-    /**
-     * Determines the authorities that need to be added to the specified user.
-     *
-     * @param user
-     * @param authorities
-     * @return
-     */
-    protected Set<Authority> determineAuthoritiesToAdd(NiFiUser user, Set<Authority> authorities) {
-        // not using copyOf since authorities may be empty and copyOf can throw an IllegalArgumentException when empty
-        Set<Authority> authoritiesToAdd = EnumSet.noneOf(Authority.class);
-        authoritiesToAdd.addAll(authorities);
-
-        // identify the authorities that need to be inserted
-        authoritiesToAdd.removeAll(user.getAuthorities());
-
-        // return the desired authorities
-        return authoritiesToAdd;
-    }
-
-    /**
-     * Determines the authorities that need to be removed from the specified
-     * user.
-     *
-     * @param user
-     * @param authorities
-     * @return
-     */
-    protected Set<Authority> determineAuthoritiesToRemove(NiFiUser user, Set<Authority> authorities) {
-        Set<Authority> authoritiesToRemove = EnumSet.copyOf(user.getAuthorities());
-
-        // identify the authorities that need to be removed
-        authoritiesToRemove.removeAll(authorities);
-
-        // return the desired authorities
-        return authoritiesToRemove;
-    }
-
-    /**
-     * Verifies the specified users account. Includes obtaining the authorities
-     * and group according to the specified authority provider.
-     *
-     * @param authorityProvider
-     * @param user
-     */
-    protected void verifyAccount(AuthorityProvider authorityProvider, NiFiUser user) {
-        // load the roles for the user
-        Set<Authority> authorities = authorityProvider.getAuthorities(user.getDn());
-
-        // update the user's authorities
-        user.getAuthorities().clear();
-        user.getAuthorities().addAll(authorities);
-
-        // get the user group
-        user.setUserGroup(authorityProvider.getGroupForUser(user.getDn()));
-
-        // update the users status in case they were previously pending or disabled
-        user.setStatus(AccountStatus.ACTIVE);
-
-        // update the users last verified time - this timestampt shouldn't be record
-        // until the both the user's authorities and group have been synced
-        Date now = new Date();
-        user.setLastVerified(now);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AddActionsAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AddActionsAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AddActionsAction.java
deleted file mode 100644
index 5a2159f..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AddActionsAction.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.admin.service.action;
-
-import java.util.Collection;
-import org.apache.nifi.action.Action;
-import org.apache.nifi.admin.dao.ActionDAO;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.authorization.AuthorityProvider;
-
-/**
- * Adds the specified actions.
- */
-public class AddActionsAction implements AdministrationAction<Void> {
-
-    private final Collection<Action> actions;
-
-    public AddActionsAction(Collection<Action> actions) {
-        this.actions = actions;
-    }
-
-    @Override
-    public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) {
-        ActionDAO actionDao = daoFactory.getActionDAO();
-
-        // add each action
-        for (Action action : actions) {
-            actionDao.createAction(action);
-        }
-
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AdministrationAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AdministrationAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AdministrationAction.java
deleted file mode 100644
index 5818ebe..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AdministrationAction.java
+++ /dev/null
@@ -1,38 +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.admin.service.action;
-
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.authorization.AuthorityProvider;
-
-/**
- * Defines the administration action. Actions are provided a DAO factory and
- * authority provider to perform a require action.
- *
- * @param <T>
- */
-public interface AdministrationAction<T> {
-
-    /**
-     * Performs an action using the specified DAOFactory and AuthorityProvider.
-     *
-     * @param daoFactory
-     * @param authorityProvider
-     * @return
-     */
-    T execute(DAOFactory daoFactory, AuthorityProvider authorityProvider);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeDownloadAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeDownloadAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeDownloadAction.java
deleted file mode 100644
index d1b994c..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeDownloadAction.java
+++ /dev/null
@@ -1,54 +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.admin.service.action;
-
-import java.util.List;
-import java.util.Map;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.DownloadAuthorization;
-import org.apache.nifi.authorization.exception.AuthorityAccessException;
-import org.apache.nifi.authorization.exception.UnknownIdentityException;
-
-/**
- * Attempts to obtain authorization to download the content with the specified
- * attributes for the specified user.
- */
-public class AuthorizeDownloadAction implements AdministrationAction<DownloadAuthorization> {
-
-    private final List<String> dnChain;
-    private final Map<String, String> attributes;
-
-    public AuthorizeDownloadAction(List<String> dnChain, Map<String, String> attributes) {
-        this.dnChain = dnChain;
-        this.attributes = attributes;
-    }
-
-    @Override
-    public DownloadAuthorization execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) {
-        try {
-            return authorityProvider.authorizeDownload(dnChain, attributes);
-        } catch (UnknownIdentityException uie) {
-            throw new AccountNotFoundException(uie.getMessage(), uie);
-        } catch (AuthorityAccessException aae) {
-            throw new AdministrationException(aae.getMessage(), aae);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeUserAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeUserAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeUserAction.java
deleted file mode 100644
index ea6973d..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeUserAction.java
+++ /dev/null
@@ -1,175 +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.admin.service.action;
-
-import java.util.Calendar;
-import java.util.Date;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.admin.service.AccountDisabledException;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.admin.service.AccountPendingException;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.exception.AuthorityAccessException;
-import org.apache.nifi.authorization.exception.UnknownIdentityException;
-import org.apache.nifi.security.util.CertificateUtils;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUser;
-
-/**
- *
- */
-public class AuthorizeUserAction extends AbstractUserAction<NiFiUser> {
-
-    private final String dn;
-    private final int cacheDurationSeconds;
-
-    public AuthorizeUserAction(String dn, int cacheDurationSeconds) {
-        this.dn = dn;
-        this.cacheDurationSeconds = cacheDurationSeconds;
-    }
-
-    @Override
-    public NiFiUser execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
-        UserDAO userDao = daoFactory.getUserDAO();
-
-        // get the user
-        NiFiUser user = userDao.findUserByDn(dn);
-
-        // verify the user was found
-        if (user == null) {
-            // determine whether this users exists
-            boolean doesDnExist = false;
-            try {
-                doesDnExist = authorityProvider.doesDnExist(dn);
-            } catch (AuthorityAccessException aae) {
-                throw new AdministrationException(String.format("Unable to access authority details: %s", aae.getMessage()), aae);
-            }
-
-            // if the authority provider has the details for this user, create the account
-            if (doesDnExist) {
-                // create the user
-                user = new NiFiUser();
-                user.setDn(dn);
-                user.setUserName(CertificateUtils.extractUsername(dn));
-                user.setJustification("User details specified by authority provider.");
-
-                try {
-                    // verify the users account
-                    verifyAccount(authorityProvider, user);
-
-                    // get the date used for verification
-                    Date now = user.getLastVerified();
-
-                    // update the last accessed field
-                    user.setLastAccessed(now);
-                    user.setCreation(now);
-
-                    // create the new user account
-                    CreateUserAction createUser = new CreateUserAction(user);
-                    createUser.execute(daoFactory, authorityProvider);
-                } catch (UnknownIdentityException uie) {
-                    // strange since the provider just reported this dn existed but handleing anyways...
-                    throw new AccountNotFoundException(String.format("Unable to verify access for %s.", dn));
-                } catch (AuthorityAccessException aae) {
-                    throw new AdministrationException(String.format("Unable to access authority details: %s", aae.getMessage()), aae);
-                }
-            } else {
-                throw new AccountNotFoundException(String.format("Unable to verify access for %s.", dn));
-            }
-        } else {
-            Throwable providerError = null;
-
-            // verify the users account if necessary
-            if (isAccountVerificationRequired(user)) {
-                try {
-                    // verify the users account
-                    verifyAccount(authorityProvider, user);
-
-                    // update the last accessed field
-                    user.setLastAccessed(user.getLastVerified());
-                } catch (UnknownIdentityException uie) {
-                    // check the account status before attempting to update the account - depending on the account
-                    // status we might not need to update the account
-                    checkAccountStatus(user);
-
-                    // the user is currently active and they were not found in the providers - disable the account...
-                    user.setStatus(AccountStatus.DISABLED);
-
-                    // record the exception
-                    providerError = uie;
-                } catch (AuthorityAccessException aae) {
-                    throw new AdministrationException(String.format("Unable to access authority details: %s", aae.getMessage()), aae);
-                }
-            } else {
-                // verfiy the users account status before allowing access.
-                checkAccountStatus(user);
-
-                // update the users last accessed time
-                user.setLastAccessed(new Date());
-            }
-
-            // persist the user's updates
-            UpdateUserCacheAction updateUser = new UpdateUserCacheAction(user);
-            updateUser.execute(daoFactory, authorityProvider);
-
-            // persist the user's authorities
-            UpdateUserAuthoritiesCacheAction updateUserAuthorities = new UpdateUserAuthoritiesCacheAction(user);
-            updateUserAuthorities.execute(daoFactory, authorityProvider);
-
-            if (providerError != null) {
-                throw new AccountDisabledException(String.format("User credentials for %s were not found. This account has been disabled.", user.getDn()), providerError);
-            }
-        }
-
-        return user;
-    }
-
-    /**
-     * Determines if account verification is required.
-     *
-     * @return
-     */
-    private boolean isAccountVerificationRequired(NiFiUser user) {
-        // accounts that have never been verified obviously needs to be re-verified
-        if (user.getLastVerified() == null) {
-            return true;
-        }
-
-        // create a calendar and substract the threshold - anything
-        // before this time will need to be re-verified
-        Calendar calendar = Calendar.getInstance();
-        calendar.add(Calendar.SECOND, -cacheDurationSeconds);
-
-        return user.getLastVerified().before(calendar.getTime());
-    }
-
-    /**
-     * Checks the account status of the specified user.
-     *
-     * @param user
-     */
-    private void checkAccountStatus(NiFiUser user) {
-        if (AccountStatus.DISABLED.equals(user.getStatus())) {
-            throw new AccountDisabledException(String.format("Account for %s is disabled.", user.getDn()));
-        } else if (AccountStatus.PENDING.equals(user.getStatus())) {
-            throw new AccountPendingException(String.format("Account for %s is pending.", user.getDn()));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/CreateUserAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/CreateUserAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/CreateUserAction.java
deleted file mode 100644
index 3833abb..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/CreateUserAction.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.admin.service.action;
-
-import java.util.Set;
-import org.apache.nifi.admin.dao.AuthorityDAO;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.authorization.Authority;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.user.NiFiUser;
-
-/**
- * Action for creating a NiFiUser account.
- */
-public class CreateUserAction extends AbstractUserAction<Void> {
-
-    private final NiFiUser user;
-
-    public CreateUserAction(NiFiUser user) {
-        this.user = user;
-    }
-
-    @Override
-    public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
-        UserDAO userDao = daoFactory.getUserDAO();
-        AuthorityDAO authorityDao = daoFactory.getAuthorityDAO();
-
-        // create the user entry
-        userDao.createUser(user);
-
-        // create the authorities
-        Set<Authority> authorities = user.getAuthorities();
-        authorityDao.createAuthorities(authorities, user.getId());
-
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DeleteUserAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DeleteUserAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DeleteUserAction.java
deleted file mode 100644
index f93e97e..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DeleteUserAction.java
+++ /dev/null
@@ -1,68 +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.admin.service.action;
-
-import org.apache.nifi.admin.dao.AuthorityDAO;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUser;
-
-/**
- *
- */
-public class DeleteUserAction implements AdministrationAction<Void> {
-
-    private final String userId;
-
-    /**
-     * Creates a new transactions for deleting the specified user.
-     *
-     * @param userId
-     */
-    public DeleteUserAction(String userId) {
-        this.userId = userId;
-    }
-
-    @Override
-    public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
-        final AuthorityDAO authorityDAO = daoFactory.getAuthorityDAO();
-        final UserDAO userDAO = daoFactory.getUserDAO();
-
-        // find the user and ensure they are currently revoked
-        final NiFiUser user = userDAO.findUserById(userId);
-
-        // ensure the user was found
-        if (user == null) {
-            throw new AccountNotFoundException(String.format("Unable to find account with ID %s.", userId));
-        }
-
-        // ensure the user is in the appropriate state
-        if (AccountStatus.ACTIVE.equals(user.getStatus())) {
-            throw new IllegalStateException(String.format("An active user cannot be removed. Revoke user access before attempting to remove."));
-        }
-
-        // remove the user and their authorities
-        authorityDAO.deleteAuthorities(userId);
-        userDAO.deleteUser(userId);
-
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserAction.java
deleted file mode 100644
index c31f107..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserAction.java
+++ /dev/null
@@ -1,76 +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.admin.service.action;
-
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.exception.AuthorityAccessException;
-import org.apache.nifi.authorization.exception.UnknownIdentityException;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUser;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- *
- */
-public class DisableUserAction implements AdministrationAction<NiFiUser> {
-
-    private static final Logger logger = LoggerFactory.getLogger(DisableUserAction.class);
-
-    private final String id;
-
-    public DisableUserAction(String id) {
-        this.id = id;
-    }
-
-    @Override
-    public NiFiUser execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
-        UserDAO userDao = daoFactory.getUserDAO();
-
-        // get the user
-        NiFiUser user = userDao.findUserById(id);
-
-        // ensure the user exists
-        if (user == null) {
-            throw new AccountNotFoundException(String.format("Unable to find account with ID %s.", id));
-        }
-
-        // update the account
-        user.setStatus(AccountStatus.DISABLED);
-        user.setUserGroup(null);
-
-        // update the user locally
-        userDao.updateUser(user);
-
-        try {
-            // revoke the user in the authority provider
-            authorityProvider.revokeUser(user.getDn());
-        } catch (UnknownIdentityException uie) {
-            // user identity is not known
-            logger.info(String.format("User %s has already been removed from the authority provider.", user.getDn()));
-        } catch (AuthorityAccessException aae) {
-            throw new AdministrationException(String.format("Unable to revoke user '%s': %s", user.getDn(), aae.getMessage()), aae);
-        }
-
-        return user;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserGroupAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserGroupAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserGroupAction.java
deleted file mode 100644
index 385fce6..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserGroupAction.java
+++ /dev/null
@@ -1,69 +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.admin.service.action;
-
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.exception.AuthorityAccessException;
-import org.apache.nifi.authorization.exception.UnknownIdentityException;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUserGroup;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- *
- */
-public class DisableUserGroupAction implements AdministrationAction<NiFiUserGroup> {
-
-    private static final Logger logger = LoggerFactory.getLogger(DisableUserGroupAction.class);
-
-    private final String group;
-
-    public DisableUserGroupAction(final String group) {
-        this.group = group;
-    }
-
-    @Override
-    public NiFiUserGroup execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
-        final NiFiUserGroup userGroup = new NiFiUserGroup();
-
-        final UserDAO userDao = daoFactory.getUserDAO();
-
-        // update the user group locally
-        userDao.updateGroupStatus(group, AccountStatus.DISABLED);
-
-        // populate the group details
-        userGroup.setGroup(group);
-        userGroup.setUsers(userDao.findUsersForGroup(group));
-
-        try {
-            // revoke the user in the authority provider
-            authorityProvider.revokeGroup(group);
-        } catch (UnknownIdentityException uie) {
-            // user identity is not known
-            logger.info(String.format("User group %s has already been removed from the authority provider.", group));
-        } catch (AuthorityAccessException aae) {
-            throw new AdministrationException(String.format("Unable to revoke user group '%s': %s", group, aae.getMessage()), aae);
-        }
-
-        return userGroup;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/FindUserByDnAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/FindUserByDnAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/FindUserByDnAction.java
deleted file mode 100644
index 8e5b574..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/FindUserByDnAction.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.admin.service.action;
-
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.user.NiFiUser;
-
-/**
- *
- */
-public class FindUserByDnAction implements AdministrationAction<NiFiUser> {
-
-    private final String dn;
-
-    /**
-     * Creates a new transactions for getting a user with the specified DN.
-     *
-     * @param dn The DN of the user to obtain
-     */
-    public FindUserByDnAction(String dn) {
-        this.dn = dn;
-    }
-
-    @Override
-    public NiFiUser execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
-        // get a UserDAO
-        UserDAO userDAO = daoFactory.getUserDAO();
-
-        // return the desired user
-        return userDAO.findUserByDn(dn);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/FindUserByIdAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/FindUserByIdAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/FindUserByIdAction.java
deleted file mode 100644
index 3062a2e..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/FindUserByIdAction.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.admin.service.action;
-
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.user.NiFiUser;
-
-/**
- *
- */
-public class FindUserByIdAction implements AdministrationAction<NiFiUser> {
-
-    private final String id;
-
-    /**
-     * Creates a new transactions for getting a user with the specified id.
-     *
-     * @param id
-     */
-    public FindUserByIdAction(String id) {
-        this.id = id;
-    }
-
-    @Override
-    public NiFiUser execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
-        // get a UserDAO
-        UserDAO userDAO = daoFactory.getUserDAO();
-
-        // return the desired user
-        return userDAO.findUserById(id);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetActionAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetActionAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetActionAction.java
deleted file mode 100644
index 1dc5588..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetActionAction.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.admin.service.action;
-
-import org.apache.nifi.action.Action;
-import org.apache.nifi.admin.dao.ActionDAO;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.authorization.AuthorityProvider;
-
-/**
- * Gets the action with the specified id.
- */
-public class GetActionAction implements AdministrationAction<Action> {
-
-    private final Integer id;
-
-    public GetActionAction(Integer id) {
-        this.id = id;
-    }
-
-    @Override
-    public Action execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) {
-        ActionDAO actionDao = daoFactory.getActionDAO();
-        return actionDao.getAction(id);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetActionsAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetActionsAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetActionsAction.java
deleted file mode 100644
index 3b82d79..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetActionsAction.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.admin.service.action;
-
-import java.util.Date;
-import org.apache.nifi.admin.dao.ActionDAO;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.history.History;
-import org.apache.nifi.history.HistoryQuery;
-
-/**
- * Get all actions that match the specified query.
- */
-public class GetActionsAction implements AdministrationAction<History> {
-
-    private final HistoryQuery query;
-
-    public GetActionsAction(HistoryQuery query) {
-        this.query = query;
-    }
-
-    @Override
-    public History execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) {
-        ActionDAO actionDao = daoFactory.getActionDAO();
-
-        // find all matching history
-        History history = actionDao.findActions(query);
-        history.setLastRefreshed(new Date());
-
-        return history;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java
deleted file mode 100644
index 5ce663e..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.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.admin.service.action;
-
-import java.util.List;
-import java.util.Map;
-import org.apache.nifi.admin.dao.ActionDAO;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.history.PreviousValue;
-
-/**
- * Gets the action with the specified id.
- */
-public class GetPreviousValues implements AdministrationAction<Map<String, List<PreviousValue>>> {
-
-    private final String processorId;
-
-    public GetPreviousValues(String processorId) {
-        this.processorId = processorId;
-    }
-
-    @Override
-    public Map<String, List<PreviousValue>> execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) {
-        ActionDAO actionDao = daoFactory.getActionDAO();
-        return actionDao.getPreviousValues(processorId);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetUserGroupAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetUserGroupAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetUserGroupAction.java
deleted file mode 100644
index 5377c46..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetUserGroupAction.java
+++ /dev/null
@@ -1,50 +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.admin.service.action;
-
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.user.NiFiUserGroup;
-
-/**
- *
- */
-public class GetUserGroupAction implements AdministrationAction<NiFiUserGroup> {
-
-    private final String group;
-
-    public GetUserGroupAction(String group) {
-        this.group = group;
-    }
-
-    @Override
-    public NiFiUserGroup execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
-        final UserDAO userDAO = daoFactory.getUserDAO();
-        final NiFiUserGroup userGroup = new NiFiUserGroup();
-
-        // set the group
-        userGroup.setGroup(group);
-
-        // get the users in this group
-        userGroup.setUsers(userDAO.findUsersForGroup(group));
-
-        // return the group
-        return userGroup;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetUsersAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetUsersAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetUsersAction.java
deleted file mode 100644
index 42d180e..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetUsersAction.java
+++ /dev/null
@@ -1,39 +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.admin.service.action;
-
-import java.util.Collection;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.user.NiFiUser;
-
-/**
- *
- */
-public class GetUsersAction implements AdministrationAction<Collection<NiFiUser>> {
-
-    @Override
-    public Collection<NiFiUser> execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
-        // get a UserDAO
-        UserDAO userDAO = daoFactory.getUserDAO();
-
-        // return the desired user
-        return userDAO.findUsers();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/HasPendingUserAccounts.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/HasPendingUserAccounts.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/HasPendingUserAccounts.java
deleted file mode 100644
index 3325642..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/HasPendingUserAccounts.java
+++ /dev/null
@@ -1,34 +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.admin.service.action;
-
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.authorization.AuthorityProvider;
-
-/**
- * Action for creating a NiFiUser account.
- */
-public class HasPendingUserAccounts extends AbstractUserAction<Boolean> {
-
-    @Override
-    public Boolean execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
-        UserDAO userDao = daoFactory.getUserDAO();
-        return userDao.hasPendingUserAccounts();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/InvalidateUserAccountAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/InvalidateUserAccountAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/InvalidateUserAccountAction.java
deleted file mode 100644
index 14596b2..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/InvalidateUserAccountAction.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.admin.service.action;
-
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.user.NiFiUser;
-
-/**
- * Invalidates a user account.
- */
-public class InvalidateUserAccountAction implements AdministrationAction<Void> {
-
-    private final String id;
-
-    public InvalidateUserAccountAction(String id) {
-        this.id = id;
-    }
-
-    @Override
-    public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
-        UserDAO userDao = daoFactory.getUserDAO();
-
-        // get the current user details
-        NiFiUser user = userDao.findUserById(id);
-
-        // ensure the user exists
-        if (user == null) {
-            throw new AccountNotFoundException(String.format("Unable to find account with ID %s.", id));
-        }
-
-        // invalidate the user account
-        user.setLastVerified(null);
-
-        // create the user entry
-        userDao.updateUser(user);
-
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/InvalidateUserGroupAccountsAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/InvalidateUserGroupAccountsAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/InvalidateUserGroupAccountsAction.java
deleted file mode 100644
index 0cb7e14..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/InvalidateUserGroupAccountsAction.java
+++ /dev/null
@@ -1,45 +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.admin.service.action;
-
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.authorization.AuthorityProvider;
-
-/**
- * Invalidates a user account.
- */
-public class InvalidateUserGroupAccountsAction implements AdministrationAction<Void> {
-
-    private final String group;
-
-    public InvalidateUserGroupAccountsAction(String group) {
-        this.group = group;
-    }
-
-    @Override
-    public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
-        UserDAO userDao = daoFactory.getUserDAO();
-
-        // create the user entry
-        userDao.updateGroupVerification(group, null);
-
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/PurgeActionsAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/PurgeActionsAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/PurgeActionsAction.java
deleted file mode 100644
index b5a2883..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/PurgeActionsAction.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.admin.service.action;
-
-import java.util.Date;
-import org.apache.nifi.action.Action;
-import org.apache.nifi.admin.dao.ActionDAO;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.authorization.AuthorityProvider;
-
-/**
- * Purges actions up to a specified end date.
- */
-public class PurgeActionsAction implements AdministrationAction<Void> {
-
-    private final Date end;
-    private final Action purgeAction;
-
-    public PurgeActionsAction(Date end, Action purgeAction) {
-        this.end = end;
-        this.purgeAction = purgeAction;
-    }
-
-    @Override
-    public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) {
-        ActionDAO actionDao = daoFactory.getActionDAO();
-
-        // remove the corresponding actions
-        actionDao.deleteActions(end);
-
-        // create a purge action
-        actionDao.createAction(purgeAction);
-
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/RequestUserAccountAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/RequestUserAccountAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/RequestUserAccountAction.java
deleted file mode 100644
index 3dce6d9..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/RequestUserAccountAction.java
+++ /dev/null
@@ -1,67 +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.admin.service.action;
-
-import java.util.Date;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.security.util.CertificateUtils;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUser;
-
-/**
- *
- */
-public class RequestUserAccountAction implements AdministrationAction<NiFiUser> {
-
-    private final String dn;
-    private final String justification;
-
-    public RequestUserAccountAction(String dn, String justification) {
-        this.dn = dn;
-        this.justification = justification;
-    }
-
-    @Override
-    public NiFiUser execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
-        UserDAO userDao = daoFactory.getUserDAO();
-
-        // determine if this user already exists
-        NiFiUser user = userDao.findUserByDn(dn);
-        if (user != null) {
-            throw new IllegalArgumentException(String.format("User account for %s already exists.", dn));
-        }
-
-        // create the user
-        user = new NiFiUser();
-        user.setDn(dn);
-        user.setUserName(CertificateUtils.extractUsername(dn));
-        user.setJustification(justification);
-        user.setStatus(AccountStatus.PENDING);
-
-        // update user timestamps
-        Date now = new Date();
-        user.setCreation(now);
-
-        // create the new user account
-        userDao.createUser(user);
-
-        return user;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/SeedUserAccountsAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/SeedUserAccountsAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/SeedUserAccountsAction.java
deleted file mode 100644
index 72d68db..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/SeedUserAccountsAction.java
+++ /dev/null
@@ -1,164 +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.admin.service.action;
-
-import java.util.HashSet;
-import java.util.Set;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.authorization.Authority;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.exception.AuthorityAccessException;
-import org.apache.nifi.authorization.exception.UnknownIdentityException;
-import org.apache.nifi.security.util.CertificateUtils;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUser;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Seeds the user accounts. This action is performed at start up because it
- * takes the users specified in the authority provider and makes them available
- * to be seen in the UI. This happens because the UI loads the users from the
- * cache. Without pre loading the users, the table in the UI would only show a
- * given user once they have visited the application.
- */
-public class SeedUserAccountsAction extends AbstractUserAction<Void> {
-
-    private static final Logger logger = LoggerFactory.getLogger(SeedUserAccountsAction.class);
-
-    @Override
-    public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
-        UserDAO userDao = daoFactory.getUserDAO();
-        Set<String> authorizedDns = new HashSet<>();
-
-        // get the current user cache
-        final Set<NiFiUser> existingUsers;
-        try {
-            existingUsers = userDao.findUsers();
-        } catch (Exception e) {
-            // unable to access local cache... start up failure
-            logger.error(String.format("Unable to get existing user base. Cannot proceed until these users can be "
-                    + "verified against the current authority provider: %s", e));
-            throw new AdministrationException(e);
-        }
-
-        try {
-            // all users for all roles
-            for (final Authority authority : Authority.values()) {
-                authorizedDns.addAll(authorityProvider.getUsers(authority));
-            }
-        } catch (AuthorityAccessException aae) {
-            // unable to access the authority provider... honor the cache
-            logger.warn("Unable to access authority provider due to " + aae);
-            return null;
-        }
-
-        final Set<NiFiUser> accountsToRevoke = new HashSet<>(existingUsers);
-
-        // persist the users
-        for (String dn : authorizedDns) {
-            NiFiUser user = null;
-            try {
-                // locate the user for this dn
-                user = userDao.findUserByDn(dn);
-                boolean newAccount = false;
-
-                // if the user does not exist, create a new account
-                if (user == null) {
-                    logger.info(String.format("Creating user account: %s", dn));
-                    newAccount = true;
-
-                    // create the user
-                    user = new NiFiUser();
-                    user.setDn(dn);
-                    user.setUserName(CertificateUtils.extractUsername(dn));
-                    user.setJustification("User details specified by authority provider.");
-                } else {
-                    logger.info(String.format("User account already created: %s. Updating authorities...", dn));
-                }
-
-                // verify the account
-                verifyAccount(authorityProvider, user);
-
-                // persist the account accordingly
-                if (newAccount) {
-                    CreateUserAction createUser = new CreateUserAction(user);
-                    createUser.execute(daoFactory, authorityProvider);
-                } else {
-                    // this is not a new user and we have just verified their 
-                    // account, do not revoke...
-                    accountsToRevoke.remove(user);
-
-                    // persist the user
-                    UpdateUserCacheAction updateUser = new UpdateUserCacheAction(user);
-                    updateUser.execute(daoFactory, authorityProvider);
-
-                    // persist the user's authorities
-                    UpdateUserAuthoritiesCacheAction updateUserAuthorities = new UpdateUserAuthoritiesCacheAction(user);
-                    updateUserAuthorities.execute(daoFactory, authorityProvider);
-                }
-            } catch (DataAccessException dae) {
-                if (user != null) {
-                    logger.warn(String.format("Unable to access account details in local cache for user %s: %s", user, dae.getMessage()));
-                } else {
-                    logger.warn(String.format("Unable to access account details in local cache: %s", dae.getMessage()));
-                }
-            } catch (UnknownIdentityException uie) {
-                if (user != null) {
-                    logger.warn(String.format("Unable to find account details in authority provider for user %s: %s", user, uie.getMessage()));
-                } else {
-                    logger.warn(String.format("Unable to find account details in authority provider: %s", uie.getMessage()));
-                }
-            } catch (AuthorityAccessException aae) {
-                logger.warn("Unable to access authority provider due to " + aae);
-
-                // unable to access authority provider for this user, honor the cache for now
-                accountsToRevoke.remove(user);
-            }
-        }
-
-        // remove all users that are no longer in the provider
-        for (final NiFiUser user : accountsToRevoke) {
-            // allow pending requests to remain...
-            if (AccountStatus.PENDING.equals(user.getStatus())) {
-                continue;
-            }
-
-            try {
-                logger.info(String.format("User not authorized with configured provider: %s. Disabling account...", user.getDn()));
-
-                // disable the account and reset its last verified timestamp since it was not found 
-                // in the current configured authority provider
-                user.setStatus(AccountStatus.DISABLED);
-                user.setLastVerified(null);
-
-                // update the user record
-                UpdateUserCacheAction updateUser = new UpdateUserCacheAction(user);
-                updateUser.execute(daoFactory, authorityProvider);
-            } catch (final Exception e) {
-                // unable to revoke access for someone we know is not authorized... fail start up
-                logger.error(String.format("Unable to revoke access for user %s that is no longer authorized: %s", user, e));
-                throw new AdministrationException(e);
-            }
-        }
-
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserAction.java
deleted file mode 100644
index 01eaf5f..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserAction.java
+++ /dev/null
@@ -1,69 +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.admin.service.action;
-
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.exception.AuthorityAccessException;
-import org.apache.nifi.authorization.exception.UnknownIdentityException;
-import org.apache.nifi.user.NiFiUser;
-
-/**
- *
- */
-public class UngroupUserAction extends AbstractUserAction<Void> {
-
-    private final String userId;
-
-    public UngroupUserAction(String userId) {
-        this.userId = userId;
-    }
-
-    @Override
-    public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) {
-        final UserDAO userDao = daoFactory.getUserDAO();
-
-        // get the user in question
-        final NiFiUser user = userDao.findUserById(userId);
-
-        // ensure the user exists
-        if (user == null) {
-            throw new AccountNotFoundException(String.format("Unable to find account with ID %s.", userId));
-        }
-
-        // set the user group
-        user.setUserGroup(null);
-
-        // update the user locally
-        userDao.updateUser(user);
-
-        try {
-            // update the authority provider
-            authorityProvider.ungroupUser(user.getDn());
-        } catch (UnknownIdentityException uie) {
-            throw new AccountNotFoundException(String.format("Unable to ungroup user '%s': %s", user.getDn(), uie.getMessage()), uie);
-        } catch (AuthorityAccessException aae) {
-            throw new AdministrationException(String.format("Unable to ungroup user '%s': %s", user.getDn(), aae.getMessage()), aae);
-        }
-
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserGroupAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserGroupAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserGroupAction.java
deleted file mode 100644
index fa24fbe..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserGroupAction.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.admin.service.action;
-
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.exception.AuthorityAccessException;
-import org.apache.nifi.authorization.exception.UnknownIdentityException;
-
-/**
- *
- */
-public class UngroupUserGroupAction extends AbstractUserAction<Void> {
-
-    private final String group;
-
-    public UngroupUserGroupAction(String group) {
-        this.group = group;
-    }
-
-    @Override
-    public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) {
-        final UserDAO userDao = daoFactory.getUserDAO();
-
-        // update the user locally
-        userDao.ungroup(group);
-
-        try {
-            // update the authority provider
-            authorityProvider.ungroup(group);
-        } catch (UnknownIdentityException uie) {
-            throw new AccountNotFoundException(String.format("Unable to ungroup '%s': %s", group, uie.getMessage()), uie);
-        } catch (AuthorityAccessException aae) {
-            throw new AdministrationException(String.format("Unable to ungroup '%s': %s", group, aae.getMessage()), aae);
-        }
-
-        return null;
-    }
-
-}


[51/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
Reworked overall directory structure to make releasing nifi vs maven plugis easier


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

Branch: refs/heads/develop
Commit: 300952a984dd2d350438a404f4e04884f7a58b53
Parents: ad74a43
Author: joewitt <jo...@apache.org>
Authored: Thu Jan 15 21:11:07 2015 -0500
Committer: joewitt <jo...@apache.org>
Committed: Thu Jan 15 21:11:07 2015 -0500

----------------------------------------------------------------------
 assembly/pom.xml                                |  458 ---
 assembly/src/main/assembly/dependencies.xml     |  140 -
 commons/data-provenance-utils/.gitignore        |    2 -
 commons/data-provenance-utils/pom.xml           |   40 -
 .../nifi/provenance/AsyncLineageSubmission.java |   87 -
 .../nifi/provenance/AsyncQuerySubmission.java   |   81 -
 .../nifi/provenance/NamedSearchableField.java   |   95 -
 .../nifi/provenance/SearchableFieldParser.java  |   53 -
 .../nifi/provenance/SearchableFields.java       |   84 -
 .../nifi/provenance/StandardLineageResult.java  |  324 --
 .../StandardProvenanceEventRecord.java          |  752 ----
 .../nifi/provenance/StandardQueryResult.java    |  168 -
 .../nifi/provenance/lineage/EdgeNode.java       |   74 -
 .../nifi/provenance/lineage/EventNode.java      |  109 -
 .../provenance/lineage/FlowFileLineage.java     |   76 -
 .../nifi/provenance/lineage/FlowFileNode.java   |   83 -
 commons/flowfile-packager/pom.xml               |   41 -
 .../org/apache/nifi/util/FlowFilePackager.java  |   28 -
 .../apache/nifi/util/FlowFilePackagerV1.java    |  104 -
 .../apache/nifi/util/FlowFilePackagerV2.java    |  146 -
 .../apache/nifi/util/FlowFilePackagerV3.java    |   93 -
 .../apache/nifi/util/FlowFileUnpackager.java    |   30 -
 .../apache/nifi/util/FlowFileUnpackagerV1.java  |  155 -
 .../apache/nifi/util/FlowFileUnpackagerV2.java  |  143 -
 .../apache/nifi/util/FlowFileUnpackagerV3.java  |  161 -
 .../java/org/apache/nifi/util/Unpackage.java    |  119 -
 .../nifi/util/TestPackageUnpackageV3.java       |   56 -
 commons/nifi-expression-language/pom.xml        |   56 -
 .../language/antlr/AttributeExpressionLexer.g   |  210 -
 .../language/antlr/AttributeExpressionParser.g  |  139 -
 .../output/AttributeExpressionLexer.tokens      |   88 -
 .../expression/language/EmptyPreparedQuery.java |   62 -
 .../language/InvalidPreparedQuery.java          |   71 -
 .../expression/language/PreparedQuery.java      |   39 -
 .../attribute/expression/language/Query.java    | 1186 ------
 .../language/StandardAttributeExpression.java   |   65 -
 .../StandardExpressionLanguageCompiler.java     |   58 -
 .../language/StandardPreparedQuery.java         |   83 -
 .../language/evaluation/BooleanEvaluator.java   |   32 -
 .../language/evaluation/BooleanQueryResult.java |   43 -
 .../language/evaluation/DateEvaluator.java      |   34 -
 .../language/evaluation/DateQueryResult.java    |   45 -
 .../language/evaluation/Evaluator.java          |   32 -
 .../language/evaluation/NumberEvaluator.java    |   33 -
 .../language/evaluation/NumberQueryResult.java  |   43 -
 .../language/evaluation/QueryResult.java        |   26 -
 .../language/evaluation/StringEvaluator.java    |   32 -
 .../language/evaluation/StringQueryResult.java  |   43 -
 .../evaluation/cast/BooleanCastEvaluator.java   |   50 -
 .../evaluation/cast/DateCastEvaluator.java      |  117 -
 .../evaluation/cast/NumberCastEvaluator.java    |   72 -
 .../evaluation/cast/StringCastEvaluator.java    |   49 -
 .../evaluation/functions/AndEvaluator.java      |   60 -
 .../evaluation/functions/AppendEvaluator.java   |   50 -
 .../functions/AttributeEvaluator.java           |   45 -
 .../evaluation/functions/ContainsEvaluator.java |   53 -
 .../functions/DateToNumberEvaluator.java        |   50 -
 .../evaluation/functions/DivideEvaluator.java   |   57 -
 .../evaluation/functions/EndsWithEvaluator.java |   53 -
 .../evaluation/functions/EqualsEvaluator.java   |   89 -
 .../functions/EqualsIgnoreCaseEvaluator.java    |   60 -
 .../evaluation/functions/FindEvaluator.java     |   72 -
 .../evaluation/functions/FormatEvaluator.java   |   61 -
 .../functions/GreaterThanEvaluator.java         |   60 -
 .../functions/GreaterThanOrEqualEvaluator.java  |   60 -
 .../evaluation/functions/HostnameEvaluator.java |   58 -
 .../evaluation/functions/IPEvaluator.java       |   46 -
 .../evaluation/functions/IndexOfEvaluator.java  |   53 -
 .../evaluation/functions/IsEmptyEvaluator.java  |   43 -
 .../evaluation/functions/IsNullEvaluator.java   |   45 -
 .../functions/LastIndexOfEvaluator.java         |   53 -
 .../evaluation/functions/LengthEvaluator.java   |   46 -
 .../evaluation/functions/LessThanEvaluator.java |   60 -
 .../functions/LessThanOrEqualEvaluator.java     |   60 -
 .../evaluation/functions/MatchesEvaluator.java  |   71 -
 .../evaluation/functions/MinusEvaluator.java    |   57 -
 .../evaluation/functions/ModEvaluator.java      |   57 -
 .../evaluation/functions/MultiplyEvaluator.java |   57 -
 .../evaluation/functions/NotEvaluator.java      |   49 -
 .../evaluation/functions/NotNullEvaluator.java  |   45 -
 .../evaluation/functions/NowEvaluator.java      |   39 -
 .../functions/NumberToDateEvaluator.java        |   52 -
 .../functions/OneUpSequenceEvaluator.java       |   41 -
 .../evaluation/functions/OrEvaluator.java       |   60 -
 .../evaluation/functions/PlusEvaluator.java     |   57 -
 .../evaluation/functions/PrependEvaluator.java  |   50 -
 .../functions/ReplaceAllEvaluator.java          |   55 -
 .../functions/ReplaceEmptyEvaluator.java        |   50 -
 .../evaluation/functions/ReplaceEvaluator.java  |   55 -
 .../functions/ReplaceNullEvaluator.java         |   47 -
 .../functions/StartsWithEvaluator.java          |   53 -
 .../functions/StringToDateEvaluator.java        |   65 -
 .../functions/SubstringAfterEvaluator.java      |   59 -
 .../functions/SubstringAfterLastEvaluator.java  |   55 -
 .../functions/SubstringBeforeEvaluator.java     |   58 -
 .../functions/SubstringBeforeLastEvaluator.java |   55 -
 .../functions/SubstringEvaluator.java           |   65 -
 .../evaluation/functions/ToLowerEvaluator.java  |   45 -
 .../evaluation/functions/ToNumberEvaluator.java |   46 -
 .../evaluation/functions/ToRadixEvaluator.java  |   77 -
 .../evaluation/functions/ToStringEvaluator.java |   45 -
 .../evaluation/functions/ToUpperEvaluator.java  |   45 -
 .../evaluation/functions/TrimEvaluator.java     |   45 -
 .../functions/UrlDecodeEvaluator.java           |   55 -
 .../functions/UrlEncodeEvaluator.java           |   55 -
 .../evaluation/functions/UuidEvaluator.java     |   39 -
 .../literals/BooleanLiteralEvaluator.java       |   44 -
 .../literals/NumberLiteralEvaluator.java        |   44 -
 .../literals/StringLiteralEvaluator.java        |   77 -
 .../evaluation/reduce/CountEvaluator.java       |   56 -
 .../evaluation/reduce/JoinEvaluator.java        |   59 -
 .../evaluation/reduce/ReduceEvaluator.java      |   23 -
 .../selection/AllAttributesEvaluator.java       |   68 -
 .../selection/AnyAttributeEvaluator.java        |   68 -
 .../AnyMatchingAttributeEvaluator.java          |   21 -
 .../selection/DelineatedAttributeEvaluator.java |   83 -
 .../evaluation/selection/MappingEvaluator.java  |   61 -
 .../selection/MultiAttributeEvaluator.java      |   24 -
 .../selection/MultiMatchAttributeEvaluator.java |   82 -
 .../selection/MultiNamedAttributeEvaluator.java |   64 -
 .../AttributeExpressionLanguageException.java   |   34 -
 ...ibuteExpressionLanguageParsingException.java |   34 -
 .../exception/IllegalAttributeException.java    |   29 -
 .../expression/language/TestQuery.java          | 1143 ------
 .../language/TestStandardPreparedQuery.java     |   92 -
 commons/nifi-logging-utils/pom.xml              |   36 -
 .../java/org/apache/nifi/logging/NiFiLog.java   |  367 --
 commons/nifi-properties/.gitignore              |    3 -
 commons/nifi-properties/pom.xml                 |   29 -
 .../org/apache/nifi/util/NiFiProperties.java    |  876 -----
 .../java/org/apache/nifi/util/StringUtils.java  |   66 -
 commons/nifi-security-utils/pom.xml             |   40 -
 .../nifi/security/util/CertificateUtils.java    |  158 -
 .../nifi/security/util/EncryptionMethod.java    |   84 -
 .../apache/nifi/security/util/KeystoreType.java |   26 -
 .../nifi/security/util/SecurityStoreTypes.java  |  144 -
 .../nifi/security/util/SslContextFactory.java   |  180 -
 commons/nifi-socket-utils/pom.xml               |   60 -
 .../nifi/io/nio/AbstractChannelReader.java      |  166 -
 .../java/org/apache/nifi/io/nio/BufferPool.java |  114 -
 .../apache/nifi/io/nio/ChannelDispatcher.java   |  160 -
 .../org/apache/nifi/io/nio/ChannelListener.java |  228 --
 .../nifi/io/nio/DatagramChannelReader.java      |   59 -
 .../apache/nifi/io/nio/SocketChannelReader.java |   55 -
 .../io/nio/consumer/AbstractStreamConsumer.java |  132 -
 .../nifi/io/nio/consumer/StreamConsumer.java    |   80 -
 .../io/nio/consumer/StreamConsumerFactory.java  |   27 -
 .../nifi/io/socket/SSLContextFactory.java       |  102 -
 .../io/socket/ServerSocketConfiguration.java    |   83 -
 .../nifi/io/socket/SocketConfiguration.java     |  116 -
 .../apache/nifi/io/socket/SocketListener.java   |  211 -
 .../org/apache/nifi/io/socket/SocketUtils.java  |  169 -
 .../socket/multicast/DiscoverableService.java   |   43 -
 .../multicast/DiscoverableServiceImpl.java      |   78 -
 .../multicast/MulticastConfiguration.java       |   99 -
 .../io/socket/multicast/MulticastListener.java  |  193 -
 .../multicast/MulticastServiceDiscovery.java    |   34 -
 .../multicast/MulticastServicesBroadcaster.java |   33 -
 .../socket/multicast/MulticastTimeToLive.java   |   50 -
 .../io/socket/multicast/MulticastUtils.java     |  109 -
 .../io/socket/multicast/ServiceDiscovery.java   |   31 -
 .../socket/multicast/ServicesBroadcaster.java   |   56 -
 .../apache/nifi/io/nio/example/ServerMain.java  |  141 -
 .../apache/nifi/io/nio/example/TCPClient.java   |   86 -
 .../apache/nifi/io/nio/example/UDPClient.java   |   51 -
 .../io/nio/example/UselessStreamConsumer.java   |   43 -
 .../src/test/resources/log4j.xml                |   36 -
 commons/nifi-utils/.gitignore                   |    8 -
 commons/nifi-utils/pom.xml                      |   33 -
 .../flowfile/attributes/CoreAttributes.java     |   72 -
 .../attributes/FlowFileAttributeKey.java        |   21 -
 .../nifi/remote/StandardVersionNegotiator.java  |   81 -
 .../apache/nifi/remote/VersionNegotiator.java   |   65 -
 .../TransmissionDisabledException.java          |   25 -
 .../nifi/remote/io/CompressionInputStream.java  |  184 -
 .../nifi/remote/io/CompressionOutputStream.java |  147 -
 .../remote/io/InterruptableInputStream.java     |  117 -
 .../remote/io/InterruptableOutputStream.java    |   81 -
 .../remote/io/socket/BufferStateManager.java    |  111 -
 .../io/socket/SocketChannelInputStream.java     |  157 -
 .../io/socket/SocketChannelOutputStream.java    |  113 -
 .../remote/io/socket/ssl/SSLSocketChannel.java  |  602 ---
 .../socket/ssl/SSLSocketChannelInputStream.java |   62 -
 .../ssl/SSLSocketChannelOutputStream.java       |   53 -
 .../nifi/stream/io/BufferedInputStream.java     |   37 -
 .../nifi/stream/io/BufferedOutputStream.java    |  140 -
 .../nifi/stream/io/ByteArrayInputStream.java    |  250 --
 .../nifi/stream/io/ByteArrayOutputStream.java   |  250 --
 .../nifi/stream/io/ByteCountingInputStream.java |  104 -
 .../stream/io/ByteCountingOutputStream.java     |   63 -
 .../apache/nifi/stream/io/DataOutputStream.java |  417 --
 .../apache/nifi/stream/io/GZIPOutputStream.java |   41 -
 .../stream/io/LeakyBucketStreamThrottler.java   |  331 --
 .../nifi/stream/io/NonCloseableInputStream.java |   56 -
 .../stream/io/NonCloseableOutputStream.java     |   51 -
 .../apache/nifi/stream/io/NullOutputStream.java |   46 -
 .../apache/nifi/stream/io/StreamThrottler.java  |   33 -
 .../org/apache/nifi/stream/io/StreamUtils.java  |  257 --
 .../apache/nifi/stream/io/ZipOutputStream.java  |   38 -
 .../exception/BytePatternNotFoundException.java |   28 -
 .../io/util/NonThreadSafeCircularBuffer.java    |   69 -
 .../org/apache/nifi/util/BooleanHolder.java     |   25 -
 .../java/org/apache/nifi/util/FormatUtils.java  |  205 -
 .../org/apache/nifi/util/IntegerHolder.java     |   54 -
 .../java/org/apache/nifi/util/LongHolder.java   |   60 -
 .../apache/nifi/util/NaiveSearchRingBuffer.java |  135 -
 .../java/org/apache/nifi/util/ObjectHolder.java |   39 -
 .../java/org/apache/nifi/util/RingBuffer.java   |  292 --
 .../java/org/apache/nifi/util/StopWatch.java    |  127 -
 .../main/java/org/apache/nifi/util/Tuple.java   |   83 -
 .../concurrency/DebugDisabledTimedLock.java     |   67 -
 .../util/concurrency/DebugEnabledTimedLock.java |  136 -
 .../util/concurrency/DebuggableTimedLock.java   |   30 -
 .../apache/nifi/util/concurrency/TimedLock.java |   59 -
 .../org/apache/nifi/util/file/FileUtils.java    |  623 ---
 .../file/monitor/CompoundUpdateMonitor.java     |  115 -
 .../util/file/monitor/LastModifiedMonitor.java  |   30 -
 .../nifi/util/file/monitor/MD5SumMonitor.java   |   51 -
 .../file/monitor/SynchronousFileWatcher.java    |  123 -
 .../nifi/util/file/monitor/UpdateMonitor.java   |   25 -
 .../org/apache/nifi/util/search/Search.java     |   57 -
 .../org/apache/nifi/util/search/SearchTerm.java |  141 -
 .../util/search/ahocorasick/AhoCorasick.java    |  155 -
 .../nifi/util/search/ahocorasick/Node.java      |   72 -
 .../util/search/ahocorasick/SearchState.java    |   63 -
 .../nifi/util/timebuffer/EntityAccess.java      |   26 -
 .../nifi/util/timebuffer/LongEntityAccess.java  |   43 -
 .../nifi/util/timebuffer/TimedBuffer.java       |  114 -
 .../nifi/util/timebuffer/TimestampedLong.java   |   35 -
 .../io/TestCompressionInputOutputStreams.java   |  153 -
 .../stream/io/TestLeakyBucketThrottler.java     |  147 -
 .../nifi/util/TestNaiveSearchRingBuffer.java    |   72 -
 .../file/monitor/TestCompoundUpdateMonitor.java |   71 -
 .../monitor/TestSynchronousFileWatcher.java     |   61 -
 .../nifi/util/timebuffer/TestRingBuffer.java    |  182 -
 .../nifi/util/timebuffer/TestTimedBuffer.java   |  106 -
 .../src/test/resources/logback-test.xml         |   32 -
 commons/nifi-web-utils/pom.xml                  |   56 -
 .../org/apache/nifi/web/util/ClientUtils.java   |  132 -
 .../nifi/web/util/ObjectMapperResolver.java     |   48 -
 .../java/org/apache/nifi/web/util/WebUtils.java |  198 -
 commons/pom.xml                                 |   44 -
 commons/processor-utilities/pom.xml             |   45 -
 .../nifi/processor/util/FlowFileFilters.java    |   65 -
 .../nifi/processor/util/SSLProperties.java      |  226 --
 .../nifi/processor/util/StandardValidators.java |  544 ---
 .../apache/nifi/processor/TestFormatUtils.java  |   40 -
 .../processor/util/TestStandardValidators.java  |   54 -
 commons/wali/.gitignore                         |    2 -
 commons/wali/pom.xml                            |   41 -
 .../org/wali/MinimalLockingWriteAheadLog.java   | 1008 -----
 commons/wali/src/main/java/org/wali/SerDe.java  |  128 -
 .../src/main/java/org/wali/SyncListener.java    |   62 -
 .../wali/src/main/java/org/wali/UpdateType.java |   49 -
 .../java/org/wali/WriteAheadRepository.java     |  122 -
 .../src/test/java/org/wali/DummyRecord.java     |   61 -
 .../test/java/org/wali/DummyRecordSerde.java    |  107 -
 .../wali/TestMinimalLockingWriteAheadLog.java   |  298 --
 maven-plugins/nar-maven-plugin/pom.xml          |   75 +
 .../src/main/java/nifi/NarMojo.java             |  613 +++
 .../resources/META-INF/plexus/components.xml    |   52 +
 maven-plugins/pom.xml                           |  346 ++
 .../execute-script-processors/pom.xml           |   81 -
 .../nifi/processors/script/ExecuteScript.java   |  566 ---
 .../apache/nifi/scripting/ConverterScript.java  |  131 -
 .../nifi/scripting/JRubyScriptFactory.java      |   46 -
 .../nifi/scripting/JavaScriptScriptFactory.java |   56 -
 .../nifi/scripting/JythonScriptFactory.java     |   45 -
 .../nifi/scripting/OutputStreamHandler.java     |   24 -
 .../org/apache/nifi/scripting/ReaderScript.java |   79 -
 .../java/org/apache/nifi/scripting/Script.java  |  303 --
 .../nifi/scripting/ScriptEngineFactory.java     |  117 -
 .../apache/nifi/scripting/ScriptFactory.java    |  269 --
 .../org/apache/nifi/scripting/WriterScript.java |   67 -
 .../org.apache.nifi.processor.Processor         |   15 -
 .../index.html                                  |  264 --
 .../processors/script/TestExecuteScript.java    |  939 -----
 .../src/test/resources/alwaysFail.js            |   24 -
 .../src/test/resources/alwaysFail.py            |   19 -
 .../src/test/resources/alwaysFail.rb            |   21 -
 .../src/test/resources/ffTest.js                |   28 -
 .../src/test/resources/ffTest.py                |   22 -
 .../src/test/resources/ffTest.rb                |   30 -
 .../src/test/resources/lib/Sub.py               |   18 -
 .../src/test/resources/lib/sub.js               |   22 -
 .../src/test/resources/lib/sub.rb               |   17 -
 .../src/test/resources/loadLocal.js             |   30 -
 .../src/test/resources/loadLocal.py             |   26 -
 .../src/test/resources/loadLocal.rb             |   29 -
 .../src/test/resources/log4j.xml                |   54 -
 .../src/test/resources/optionalValidators.js    |   28 -
 .../src/test/resources/optionalValidators.py    |   22 -
 .../src/test/resources/optionalValidators.rb    |   39 -
 .../src/test/resources/paramTest.js             |   28 -
 .../src/test/resources/paramTest.py             |   26 -
 .../src/test/resources/paramTest.rb             |   31 -
 .../src/test/resources/parseXml.js              |   36 -
 .../src/test/resources/readTest.js              |   30 -
 .../src/test/resources/readTest.py              |   32 -
 .../src/test/resources/readTest.rb              |   30 -
 .../src/test/resources/readWithParams.js        |   32 -
 .../src/test/resources/readWithParams.py        |   32 -
 .../src/test/resources/readWithParams.rb        |   33 -
 .../src/test/resources/routeTest.js             |   41 -
 .../src/test/resources/routeTest.py             |   37 -
 .../src/test/resources/routeTest.rb             |   39 -
 .../src/test/resources/simpleConverter.js       |   45 -
 .../src/test/resources/simpleConverter.py       |   60 -
 .../src/test/resources/simpleConverter.rb       |   42 -
 .../src/test/resources/writeTest.js             |   26 -
 .../src/test/resources/writeTest.py             |   22 -
 .../src/test/resources/writeTest.rb             |   32 -
 nar-bundles/execute-script-bundle/nar/pom.xml   |   36 -
 nar-bundles/execute-script-bundle/pom.xml       |   81 -
 .../framework/administration/.gitignore         |    1 -
 .../framework/administration/pom.xml            |  116 -
 .../nifi/admin/AuditDataSourceFactoryBean.java  |  222 --
 .../org/apache/nifi/admin/RepositoryUtils.java  |   91 -
 .../nifi/admin/UserDataSourceFactoryBean.java   |  247 --
 .../org/apache/nifi/admin/dao/ActionDAO.java    |   74 -
 .../org/apache/nifi/admin/dao/AuthorityDAO.java |   58 -
 .../org/apache/nifi/admin/dao/DAOFactory.java   |   29 -
 .../nifi/admin/dao/DataAccessException.java     |   39 -
 .../java/org/apache/nifi/admin/dao/UserDAO.java |  127 -
 .../nifi/admin/dao/impl/DAOFactoryImpl.java     |   51 -
 .../nifi/admin/dao/impl/StandardActionDAO.java  | 1056 -----
 .../admin/dao/impl/StandardAuthorityDAO.java    |  172 -
 .../nifi/admin/dao/impl/StandardUserDAO.java    |  634 ---
 .../admin/service/AccountDisabledException.java |   40 -
 .../admin/service/AccountNotFoundException.java |   40 -
 .../admin/service/AccountPendingException.java  |   41 -
 .../admin/service/AdministrationException.java  |   39 -
 .../apache/nifi/admin/service/AuditService.java |   76 -
 .../apache/nifi/admin/service/UserService.java  |  161 -
 .../service/action/AbstractUserAction.java      |   97 -
 .../admin/service/action/AddActionsAction.java  |   48 -
 .../service/action/AdministrationAction.java    |   38 -
 .../service/action/AuthorizeDownloadAction.java |   54 -
 .../service/action/AuthorizeUserAction.java     |  175 -
 .../admin/service/action/CreateUserAction.java  |   53 -
 .../admin/service/action/DeleteUserAction.java  |   68 -
 .../admin/service/action/DisableUserAction.java |   76 -
 .../service/action/DisableUserGroupAction.java  |   69 -
 .../service/action/FindUserByDnAction.java      |   49 -
 .../service/action/FindUserByIdAction.java      |   49 -
 .../admin/service/action/GetActionAction.java   |   41 -
 .../admin/service/action/GetActionsAction.java  |   48 -
 .../admin/service/action/GetPreviousValues.java |   43 -
 .../service/action/GetUserGroupAction.java      |   50 -
 .../admin/service/action/GetUsersAction.java    |   39 -
 .../service/action/HasPendingUserAccounts.java  |   34 -
 .../action/InvalidateUserAccountAction.java     |   58 -
 .../InvalidateUserGroupAccountsAction.java      |   45 -
 .../service/action/PurgeActionsAction.java      |   51 -
 .../action/RequestUserAccountAction.java        |   67 -
 .../service/action/SeedUserAccountsAction.java  |  164 -
 .../admin/service/action/UngroupUserAction.java |   69 -
 .../service/action/UngroupUserGroupAction.java  |   57 -
 .../admin/service/action/UpdateUserAction.java  |  124 -
 .../UpdateUserAuthoritiesCacheAction.java       |   73 -
 .../service/action/UpdateUserCacheAction.java   |   47 -
 .../service/action/UpdateUserGroupAction.java   |  171 -
 .../service/impl/StandardAuditService.java      |  230 --
 .../admin/service/impl/StandardUserService.java |  663 ----
 .../admin/service/transaction/Transaction.java  |   49 -
 .../service/transaction/TransactionBuilder.java |   25 -
 .../transaction/TransactionException.java       |   40 -
 .../transaction/impl/StandardTransaction.java   |   93 -
 .../impl/StandardTransactionBuilder.java        |   57 -
 .../AuthorityProviderFactoryBean.java           |  529 ---
 ...rdAuthorityProviderConfigurationContext.java |   50 -
 ...dAuthorityProviderInitializationContext.java |   42 -
 .../java/org/apache/nifi/history/History.java   |   56 -
 .../org/apache/nifi/history/HistoryQuery.java   |   99 -
 .../org/apache/nifi/history/PreviousValue.java  |   54 -
 .../org/apache/nifi/user/AccountStatus.java     |   47 -
 .../java/org/apache/nifi/user/NiFiUser.java     |  164 -
 .../org/apache/nifi/user/NiFiUserGroup.java     |   45 -
 .../resources/nifi-administration-context.xml   |   62 -
 .../src/main/xsd/authority-providers.xsd        |   49 -
 .../service/action/AuthorizeUserActionTest.java |  433 ---
 .../service/action/CreateUserActionTest.java    |  144 -
 .../service/action/DisableUserActionTest.java   |  171 -
 .../action/InvalidateUserAccountActionTest.java |  131 -
 .../action/RequestUserAccountActionTest.java    |  127 -
 .../action/SeedUserAccountsActionTest.java      |  263 --
 .../action/SetUserAuthoritiesActionTest.java    |  223 --
 .../framework/client-dto/.gitignore             |    6 -
 .../framework/client-dto/pom.xml                |   46 -
 .../org/apache/nifi/web/api/dto/AboutDTO.java   |   57 -
 .../org/apache/nifi/web/api/dto/BannerDTO.java  |   57 -
 .../nifi/web/api/dto/BulletinBoardDTO.java      |   63 -
 .../apache/nifi/web/api/dto/BulletinDTO.java    |  161 -
 .../nifi/web/api/dto/BulletinQueryDTO.java      |  113 -
 .../org/apache/nifi/web/api/dto/ClusterDTO.java |   61 -
 .../apache/nifi/web/api/dto/ConnectableDTO.java |  146 -
 .../apache/nifi/web/api/dto/ConnectionDTO.java  |  215 --
 .../web/api/dto/ControllerConfigurationDTO.java |  158 -
 .../apache/nifi/web/api/dto/ControllerDTO.java  |  262 --
 .../org/apache/nifi/web/api/dto/CounterDTO.java |   94 -
 .../apache/nifi/web/api/dto/CountersDTO.java    |   61 -
 .../nifi/web/api/dto/DocumentedTypeDTO.java     |   71 -
 .../apache/nifi/web/api/dto/FlowSnippetDTO.java |  141 -
 .../org/apache/nifi/web/api/dto/FunnelDTO.java  |   29 -
 .../org/apache/nifi/web/api/dto/LabelDTO.java   |   94 -
 .../nifi/web/api/dto/NiFiComponentDTO.java      |   95 -
 .../org/apache/nifi/web/api/dto/NodeDTO.java    |  188 -
 .../apache/nifi/web/api/dto/NodeEventDTO.java   |   74 -
 .../web/api/dto/NodeSystemDiagnosticsDTO.java   |   56 -
 .../org/apache/nifi/web/api/dto/PortDTO.java    |  161 -
 .../apache/nifi/web/api/dto/PositionDTO.java    |   65 -
 .../nifi/web/api/dto/PreviousValueDTO.java      |   73 -
 .../nifi/web/api/dto/ProcessGroupDTO.java       |  219 --
 .../nifi/web/api/dto/ProcessorConfigDTO.java    |  486 ---
 .../apache/nifi/web/api/dto/ProcessorDTO.java   |  181 -
 .../nifi/web/api/dto/ProcessorHistoryDTO.java   |   56 -
 .../nifi/web/api/dto/PropertyHistoryDTO.java    |   43 -
 .../nifi/web/api/dto/RelationshipDTO.java       |   69 -
 .../api/dto/RemoteProcessGroupContentsDTO.java  |   56 -
 .../nifi/web/api/dto/RemoteProcessGroupDTO.java |  279 --
 .../web/api/dto/RemoteProcessGroupPortDTO.java  |  192 -
 .../apache/nifi/web/api/dto/RevisionDTO.java    |   63 -
 .../org/apache/nifi/web/api/dto/SnippetDTO.java |  239 --
 .../nifi/web/api/dto/SystemDiagnosticsDTO.java  |  461 ---
 .../apache/nifi/web/api/dto/TemplateDTO.java    |  117 -
 .../org/apache/nifi/web/api/dto/UserDTO.java    |  177 -
 .../apache/nifi/web/api/dto/UserGroupDTO.java   |   84 -
 .../nifi/web/api/dto/action/ActionDTO.java      |  176 -
 .../nifi/web/api/dto/action/HistoryDTO.java     |   74 -
 .../web/api/dto/action/HistoryQueryDTO.java     |  144 -
 .../component/details/ComponentDetailsDTO.java  |   32 -
 .../component/details/ProcessorDetailsDTO.java  |   41 -
 .../details/RemoteProcessGroupDetailsDTO.java   |   41 -
 .../dto/action/details/ActionDetailsDTO.java    |   34 -
 .../dto/action/details/ConfigureDetailsDTO.java |   69 -
 .../dto/action/details/ConnectDetailsDTO.java   |  126 -
 .../api/dto/action/details/MoveDetailsDTO.java  |   83 -
 .../api/dto/action/details/PurgeDetailsDTO.java |   45 -
 .../web/api/dto/provenance/AttributeDTO.java    |   69 -
 .../web/api/dto/provenance/ProvenanceDTO.java   |  165 -
 .../api/dto/provenance/ProvenanceEventDTO.java  |  630 ---
 .../dto/provenance/ProvenanceOptionsDTO.java    |   43 -
 .../dto/provenance/ProvenanceRequestDTO.java    |  118 -
 .../dto/provenance/ProvenanceResultsDTO.java    |  136 -
 .../ProvenanceSearchableFieldDTO.java           |   84 -
 .../api/dto/provenance/lineage/LineageDTO.java  |  161 -
 .../provenance/lineage/LineageRequestDTO.java   |   88 -
 .../provenance/lineage/LineageResultsDTO.java   |   73 -
 .../provenance/lineage/ProvenanceLinkDTO.java   |  101 -
 .../provenance/lineage/ProvenanceNodeDTO.java   |  162 -
 .../dto/search/ComponentSearchResultDTO.java    |   85 -
 .../web/api/dto/search/NodeSearchResultDTO.java |   56 -
 .../web/api/dto/search/SearchResultsDTO.java    |  128 -
 .../dto/search/UserGroupSearchResultDTO.java    |   42 -
 .../web/api/dto/search/UserSearchResultDTO.java |   56 -
 .../dto/status/ClusterConnectionStatusDTO.java  |   89 -
 .../api/dto/status/ClusterPortStatusDTO.java    |   89 -
 .../dto/status/ClusterProcessorStatusDTO.java   |  117 -
 .../ClusterRemoteProcessGroupStatusDTO.java     |   89 -
 .../web/api/dto/status/ClusterStatusDTO.java    |   44 -
 .../api/dto/status/ClusterStatusHistoryDTO.java |   75 -
 .../web/api/dto/status/ConnectionStatusDTO.java |  198 -
 .../web/api/dto/status/ControllerStatusDTO.java |  187 -
 .../api/dto/status/NodeConnectionStatusDTO.java |   57 -
 .../web/api/dto/status/NodePortStatusDTO.java   |   57 -
 .../api/dto/status/NodeProcessorStatusDTO.java  |   57 -
 .../status/NodeRemoteProcessGroupStatusDTO.java |   57 -
 .../nifi/web/api/dto/status/NodeStatusDTO.java  |   57 -
 .../api/dto/status/NodeStatusHistoryDTO.java    |   57 -
 .../nifi/web/api/dto/status/PortStatusDTO.java  |  142 -
 .../api/dto/status/ProcessGroupStatusDTO.java   |  244 --
 .../web/api/dto/status/ProcessorStatusDTO.java  |  203 -
 .../web/api/dto/status/RemotePortStatusDTO.java |   98 -
 .../dto/status/RemoteProcessGroupStatusDTO.java |  159 -
 .../nifi/web/api/dto/status/StatusDTO.java      |   43 -
 .../web/api/dto/status/StatusDescriptorDTO.java |  101 -
 .../web/api/dto/status/StatusHistoryDTO.java    |   92 -
 .../api/dto/status/StatusHistoryDetailDTO.java  |   56 -
 .../web/api/dto/status/StatusSnapshotDTO.java   |   58 -
 .../nifi/web/api/dto/util/DateTimeAdapter.java  |   47 -
 .../nifi/web/api/dto/util/TimeAdapter.java      |   47 -
 .../nifi/web/api/dto/util/TimestampAdapter.java |   47 -
 .../apache/nifi/web/api/entity/AboutEntity.java |   45 -
 .../nifi/web/api/entity/ActionEntity.java       |   45 -
 .../nifi/web/api/entity/AuthorityEntity.java    |   60 -
 .../nifi/web/api/entity/BannerEntity.java       |   46 -
 .../web/api/entity/BulletinBoardEntity.java     |   45 -
 .../entity/ClusterConnectionStatusEntity.java   |   45 -
 .../nifi/web/api/entity/ClusterEntity.java      |   45 -
 .../web/api/entity/ClusterPortStatusEntity.java |   45 -
 .../entity/ClusterProcessorStatusEntity.java    |   45 -
 .../ClusterRemoteProcessGroupStatusEntity.java  |   45 -
 .../api/entity/ClusterSearchResultsEntity.java  |   46 -
 .../web/api/entity/ClusterStatusEntity.java     |   45 -
 .../api/entity/ClusterStatusHistoryEntity.java  |   45 -
 .../nifi/web/api/entity/ConnectionEntity.java   |   45 -
 .../nifi/web/api/entity/ConnectionsEntity.java  |   47 -
 .../entity/ControllerConfigurationEntity.java   |   45 -
 .../nifi/web/api/entity/ControllerEntity.java   |   45 -
 .../web/api/entity/ControllerStatusEntity.java  |   45 -
 .../nifi/web/api/entity/CounterEntity.java      |   45 -
 .../nifi/web/api/entity/CountersEntity.java     |   46 -
 .../org/apache/nifi/web/api/entity/Entity.java  |   43 -
 .../nifi/web/api/entity/FlowSnippetEntity.java  |   45 -
 .../nifi/web/api/entity/FunnelEntity.java       |   45 -
 .../nifi/web/api/entity/FunnelsEntity.java      |   47 -
 .../nifi/web/api/entity/HistoryEntity.java      |   45 -
 .../nifi/web/api/entity/InputPortEntity.java    |   45 -
 .../nifi/web/api/entity/InputPortsEntity.java   |   47 -
 .../apache/nifi/web/api/entity/LabelEntity.java |   45 -
 .../nifi/web/api/entity/LabelsEntity.java       |   47 -
 .../nifi/web/api/entity/LineageEntity.java      |   45 -
 .../apache/nifi/web/api/entity/NodeEntity.java  |   45 -
 .../nifi/web/api/entity/NodeStatusEntity.java   |   45 -
 .../api/entity/NodeSystemDiagnosticsEntity.java |   45 -
 .../nifi/web/api/entity/OutputPortEntity.java   |   45 -
 .../nifi/web/api/entity/OutputPortsEntity.java  |   47 -
 .../web/api/entity/PrioritizerTypesEntity.java  |   46 -
 .../nifi/web/api/entity/ProcessGroupEntity.java |   45 -
 .../api/entity/ProcessGroupStatusEntity.java    |   45 -
 .../web/api/entity/ProcessGroupsEntity.java     |   46 -
 .../nifi/web/api/entity/ProcessorEntity.java    |   45 -
 .../web/api/entity/ProcessorHistoryEntity.java  |   45 -
 .../web/api/entity/ProcessorTypesEntity.java    |   46 -
 .../nifi/web/api/entity/ProcessorsEntity.java   |   47 -
 .../nifi/web/api/entity/ProvenanceEntity.java   |   40 -
 .../web/api/entity/ProvenanceEventEntity.java   |   45 -
 .../web/api/entity/ProvenanceOptionsEntity.java |   46 -
 .../api/entity/RemoteProcessGroupEntity.java    |   45 -
 .../entity/RemoteProcessGroupPortEntity.java    |   45 -
 .../api/entity/RemoteProcessGroupsEntity.java   |   47 -
 .../web/api/entity/SearchResultsEntity.java     |   46 -
 .../nifi/web/api/entity/SnippetEntity.java      |   45 -
 .../web/api/entity/StatusHistoryEntity.java     |   45 -
 .../web/api/entity/SystemDiagnosticsEntity.java |   45 -
 .../nifi/web/api/entity/TemplateEntity.java     |   45 -
 .../nifi/web/api/entity/TemplatesEntity.java    |   63 -
 .../apache/nifi/web/api/entity/UserEntity.java  |   45 -
 .../nifi/web/api/entity/UserGroupEntity.java    |   45 -
 .../web/api/entity/UserSearchResultsEntity.java |   61 -
 .../apache/nifi/web/api/entity/UsersEntity.java |   64 -
 .../cluster-authorization-provider/.gitignore   |    1 -
 .../cluster-authorization-provider/pom.xml      |   48 -
 .../ClusterManagerAuthorizationProvider.java    |  225 --
 .../NodeAuthorizationProvider.java              |  389 --
 .../protocol/message/DoesDnExistMessage.java    |   56 -
 .../protocol/message/GetAuthoritiesMessage.java |   58 -
 .../message/GetGroupForUserMessage.java         |   55 -
 .../protocol/message/ProtocolMessage.java       |   57 -
 .../message/jaxb/JaxbProtocolUtils.java         |   42 -
 .../protocol/message/jaxb/ObjectFactory.java    |   45 -
 ....apache.nifi.authorization.AuthorityProvider |   16 -
 .../framework/cluster-protocol/.gitignore       |    1 -
 .../framework/cluster-protocol/pom.xml          |   69 -
 .../protocol/ClusterManagerProtocolSender.java  |   69 -
 .../cluster/protocol/ConnectionRequest.java     |   44 -
 .../cluster/protocol/ConnectionResponse.java    |  141 -
 .../apache/nifi/cluster/protocol/Heartbeat.java |   68 -
 .../nifi/cluster/protocol/NodeBulletins.java    |   44 -
 .../nifi/cluster/protocol/NodeIdentifier.java   |  172 -
 .../cluster/protocol/NodeProtocolSender.java    |   73 -
 .../nifi/cluster/protocol/ProtocolContext.java  |   39 -
 .../cluster/protocol/ProtocolException.java     |   40 -
 .../nifi/cluster/protocol/ProtocolHandler.java  |   44 -
 .../nifi/cluster/protocol/ProtocolListener.java |   72 -
 .../protocol/ProtocolMessageMarshaller.java     |   38 -
 .../protocol/ProtocolMessageUnmarshaller.java   |   38 -
 .../nifi/cluster/protocol/StandardDataFlow.java |  105 -
 .../UnknownServiceAddressException.java         |   39 -
 .../impl/ClusterManagerProtocolSenderImpl.java  |  245 --
 .../ClusterManagerProtocolSenderListener.java   |  118 -
 .../protocol/impl/ClusterServiceDiscovery.java  |  181 -
 .../protocol/impl/ClusterServiceLocator.java    |  229 --
 .../impl/ClusterServicesBroadcaster.java        |  182 -
 .../protocol/impl/CopyingInputStream.java       |   77 -
 .../impl/MulticastProtocolListener.java         |  204 -
 .../protocol/impl/NodeProtocolSenderImpl.java   |  171 -
 .../impl/NodeProtocolSenderListener.java        |  115 -
 .../protocol/impl/SocketProtocolListener.java   |  205 -
 .../protocol/jaxb/JaxbProtocolContext.java      |  148 -
 .../jaxb/message/AdaptedConnectionRequest.java  |   40 -
 .../jaxb/message/AdaptedConnectionResponse.java |  109 -
 .../protocol/jaxb/message/AdaptedCounter.java   |   56 -
 .../protocol/jaxb/message/AdaptedDataFlow.java  |   64 -
 .../protocol/jaxb/message/AdaptedHeartbeat.java |   66 -
 .../jaxb/message/AdaptedNodeBulletins.java      |   50 -
 .../jaxb/message/AdaptedNodeIdentifier.java     |   76 -
 .../jaxb/message/ConnectionRequestAdapter.java  |   41 -
 .../jaxb/message/ConnectionResponseAdapter.java |   55 -
 .../protocol/jaxb/message/DataFlowAdapter.java  |   50 -
 .../protocol/jaxb/message/HeartbeatAdapter.java |   54 -
 .../jaxb/message/JaxbProtocolUtils.java         |   42 -
 .../jaxb/message/NodeBulletinsAdapter.java      |   48 -
 .../jaxb/message/NodeIdentifierAdapter.java     |   51 -
 .../protocol/jaxb/message/ObjectFactory.java    |  104 -
 .../message/ConnectionRequestMessage.java       |   46 -
 .../message/ConnectionResponseMessage.java      |   66 -
 .../ControllerStartupFailureMessage.java        |   49 -
 .../protocol/message/DisconnectMessage.java     |   55 -
 .../protocol/message/ExceptionMessage.java      |   44 -
 .../protocol/message/FlowRequestMessage.java    |   46 -
 .../protocol/message/FlowResponseMessage.java   |   44 -
 .../protocol/message/HeartbeatMessage.java      |   43 -
 .../message/MulticastProtocolMessage.java       |   66 -
 .../protocol/message/NodeBulletinsMessage.java  |   43 -
 .../cluster/protocol/message/PingMessage.java   |   55 -
 .../message/PrimaryRoleAssignmentMessage.java   |   56 -
 .../protocol/message/ProtocolMessage.java       |   61 -
 .../message/ReconnectionFailureMessage.java     |   45 -
 .../message/ReconnectionRequestMessage.java     |   94 -
 .../message/ReconnectionResponseMessage.java    |   32 -
 .../message/ServiceBroadcastMessage.java        |   64 -
 .../MulticastConfigurationFactoryBean.java      |   60 -
 .../ServerSocketConfigurationFactoryBean.java   |   65 -
 .../spring/SocketConfigurationFactoryBean.java  |   66 -
 .../resources/nifi-cluster-protocol-context.xml |  110 -
 .../ClusterManagerProtocolSenderImplTest.java   |  134 -
 .../impl/ClusterServiceDiscoveryTest.java       |  135 -
 .../impl/ClusterServiceLocatorTest.java         |  121 -
 .../impl/ClusterServicesBroadcasterTest.java    |  133 -
 .../impl/MulticastProtocolListenerTest.java     |  171 -
 .../impl/NodeProtocolSenderImplTest.java        |  203 -
 .../impl/testutils/DelayedProtocolHandler.java  |   57 -
 .../testutils/ReflexiveProtocolHandler.java     |   47 -
 .../framework/cluster-web/.gitignore            |    1 -
 .../framework/cluster-web/pom.xml               |   50 -
 .../nifi/cluster/context/ClusterContext.java    |   59 -
 .../cluster/context/ClusterContextImpl.java     |   69 -
 .../context/ClusterContextThreadLocal.java      |   47 -
 .../ClusterAwareOptimisticLockingManager.java   |   96 -
 .../framework/cluster/.gitignore                |    1 -
 .../framework-bundle/framework/cluster/pom.xml  |  132 -
 .../cluster/client/MulticastTestClient.java     |  151 -
 .../org/apache/nifi/cluster/event/Event.java    |  122 -
 .../apache/nifi/cluster/event/EventManager.java |   65 -
 .../cluster/event/impl/EventManagerImpl.java    |  143 -
 .../cluster/firewall/ClusterNodeFirewall.java   |   35 -
 .../impl/FileBasedClusterNodeFirewall.java      |  207 -
 .../nifi/cluster/flow/ClusterDataFlow.java      |   45 -
 .../apache/nifi/cluster/flow/DaoException.java  |   40 -
 .../apache/nifi/cluster/flow/DataFlowDao.java   |   62 -
 .../cluster/flow/DataFlowManagementService.java |  115 -
 .../nifi/cluster/flow/PersistedFlowState.java   |   37 -
 .../nifi/cluster/flow/StaleFlowException.java   |   42 -
 .../nifi/cluster/flow/impl/DataFlowDaoImpl.java |  600 ---
 .../impl/DataFlowManagementServiceImpl.java     |  356 --
 .../nifi/cluster/manager/ClusterManager.java    |  225 --
 .../cluster/manager/HttpClusterManager.java     |  169 -
 .../cluster/manager/HttpRequestReplicator.java  |   99 -
 .../cluster/manager/HttpResponseMapper.java     |   42 -
 .../nifi/cluster/manager/NodeResponse.java      |  329 --
 .../exception/BlockedByFirewallException.java   |   60 -
 .../manager/exception/ClusterException.java     |   40 -
 .../ConnectingNodeMutableRequestException.java  |   41 -
 ...DisconnectedNodeMutableRequestException.java |   41 -
 .../exception/IllegalClusterStateException.java |   41 -
 .../exception/IllegalNodeDeletionException.java |   41 -
 .../IllegalNodeDisconnectionException.java      |   42 -
 .../IllegalNodeReconnectionException.java       |   41 -
 .../IneligiblePrimaryNodeException.java         |   41 -
 .../exception/MutableRequestException.java      |   42 -
 .../exception/NoConnectedNodesException.java    |   41 -
 .../exception/NoResponseFromNodesException.java |   42 -
 .../exception/NodeDisconnectionException.java   |   41 -
 .../exception/NodeReconnectionException.java    |   40 -
 .../PrimaryRoleAssignmentException.java         |   41 -
 .../SafeModeMutableRequestException.java        |   41 -
 .../manager/exception/UnknownNodeException.java |   41 -
 .../exception/UriConstructionException.java     |   42 -
 .../manager/impl/ClusteredEventAccess.java      |  135 -
 .../manager/impl/ClusteredReportingContext.java |  165 -
 .../manager/impl/HttpRequestReplicatorImpl.java |  531 ---
 .../manager/impl/HttpResponseMapperImpl.java    |   85 -
 .../cluster/manager/impl/WebClusterManager.java | 3620 ------------------
 .../java/org/apache/nifi/cluster/node/Node.java |  252 --
 ...anagerProtocolServiceLocatorFactoryBean.java |  116 -
 ...FileBasedClusterNodeFirewallFactoryBean.java |   58 -
 .../spring/WebClusterManagerFactoryBean.java    |  139 -
 .../reporting/ClusteredReportingTaskNode.java   |   49 -
 .../resources/nifi-cluster-manager-context.xml  |  124 -
 .../event/impl/EventManagerImplTest.java        |  119 -
 .../impl/FileBasedClusterNodeFirewallTest.java  |   98 -
 .../impl/DataFlowManagementServiceImplTest.java |  343 --
 .../impl/HttpRequestReplicatorImplTest.java     |  368 --
 .../impl/HttpResponseMapperImplTest.java        |  126 -
 .../manager/impl/TestWebClusterManager.java     |   54 -
 .../cluster/manager/testutils/HttpRequest.java  |  239 --
 .../cluster/manager/testutils/HttpResponse.java |   93 -
 .../manager/testutils/HttpResponseAction.java   |   60 -
 .../cluster/manager/testutils/HttpServer.java   |  240 --
 .../ClusterManagerProtocolSenderImplTest.java   |  133 -
 .../impl/ClusterServiceLocatorTest.java         |  119 -
 .../impl/ClusterServicesBroadcasterTest.java    |  131 -
 .../impl/MulticastProtocolListenerTest.java     |  171 -
 .../impl/NodeProtocolSenderImplTest.java        |  201 -
 .../impl/SocketProtocolListenerTest.java        |  132 -
 .../testutils/DelayedProtocolHandler.java       |   57 -
 .../testutils/ReflexiveProtocolHandler.java     |   47 -
 .../cluster/src/test/resources/logback-test.xml |   48 -
 .../apache/nifi/cluster/firewall/impl/empty.txt |    0
 .../apache/nifi/cluster/firewall/impl/ips.txt   |   12 -
 .../framework/core-api/.gitignore               |    1 -
 .../framework-bundle/framework/core-api/pom.xml |   56 -
 .../nifi/cluster/AdaptedNodeInformation.java    |   66 -
 .../nifi/cluster/ClusterNodeInformation.java    |   67 -
 .../org/apache/nifi/cluster/NodeInformant.java  |   22 -
 .../apache/nifi/cluster/NodeInformation.java    |   98 -
 .../nifi/cluster/NodeInformationAdapter.java    |   39 -
 .../apache/nifi/cluster/protocol/DataFlow.java  |   41 -
 .../apache/nifi/connectable/Connectable.java    |  291 --
 .../nifi/connectable/ConnectableType.java       |   44 -
 .../org/apache/nifi/connectable/Connection.java |   78 -
 .../org/apache/nifi/connectable/Funnel.java     |   24 -
 .../java/org/apache/nifi/connectable/Port.java  |   31 -
 .../org/apache/nifi/connectable/Position.java   |   36 -
 .../java/org/apache/nifi/connectable/Size.java  |   36 -
 .../controller/AbstractConfiguredComponent.java |  280 --
 .../apache/nifi/controller/AbstractPort.java    |  636 ---
 .../apache/nifi/controller/Availability.java    |   24 -
 .../nifi/controller/ConfiguredComponent.java    |   63 -
 .../nifi/controller/ContentAvailability.java    |   65 -
 .../org/apache/nifi/controller/Counter.java     |   32 -
 .../nifi/controller/EventBasedWorker.java       |   32 -
 .../org/apache/nifi/controller/Heartbeater.java |   22 -
 .../nifi/controller/ProcessScheduler.java       |  146 -
 .../apache/nifi/controller/ProcessorNode.java   |   80 -
 .../nifi/controller/ReportingTaskNode.java      |   56 -
 .../nifi/controller/StandardFlowFileQueue.java  | 1096 ------
 .../apache/nifi/controller/StandardFunnel.java  |  541 ---
 .../controller/ValidationContextFactory.java    |   27 -
 .../org/apache/nifi/controller/WorkerQueue.java |   36 -
 .../exception/CommunicationsException.java      |   40 -
 ...ControllerServiceAlreadyExistsException.java |   30 -
 .../ControllerServiceNotFoundException.java     |   51 -
 .../ProcessorInstantiationException.java        |   27 -
 .../exception/ProcessorLifeCycleException.java  |   30 -
 .../org/apache/nifi/controller/label/Label.java |   48 -
 .../ReportingTaskInstantiationException.java    |   31 -
 .../repository/ContentNotFoundException.java    |   48 -
 .../repository/CounterRepository.java           |   34 -
 .../controller/repository/FlowFileEvent.java    |   54 -
 .../repository/FlowFileEventRepository.java     |   50 -
 .../repository/RepositoryStatusReport.java      |   28 -
 .../service/ControllerServiceNode.java          |   40 -
 .../service/ControllerServiceProvider.java      |   47 -
 .../service/ControllerServiceReference.java     |   50 -
 .../org/apache/nifi/events/BulletinFactory.java |   52 -
 .../nifi/events/BulletinProcessingStrategy.java |   27 -
 .../apache/nifi/events/ComponentBulletin.java   |   30 -
 .../org/apache/nifi/events/SystemBulletin.java  |   30 -
 .../org/apache/nifi/groups/ProcessGroup.java    |  723 ----
 .../apache/nifi/groups/ProcessGroupCounts.java  |   66 -
 .../apache/nifi/groups/RemoteProcessGroup.java  |  255 --
 .../RemoteProcessGroupPortDescriptor.java       |   92 -
 .../org/apache/nifi/logging/LogMessage.java     |   74 -
 .../org/apache/nifi/logging/LogObserver.java    |   22 -
 .../org/apache/nifi/logging/LogRepository.java  |   67 -
 .../nifi/logging/LogRepositoryFactory.java      |   61 -
 .../java/org/apache/nifi/nar/NarCloseable.java  |   44 -
 .../nifi/nar/NarThreadContextClassLoader.java   |  188 -
 .../main/java/org/apache/nifi/remote/Peer.java  |  107 -
 .../java/org/apache/nifi/remote/PeerStatus.java |   72 -
 .../nifi/remote/PortAuthorizationResult.java    |   25 -
 .../nifi/remote/RemoteAuthorizationState.java   |   27 -
 .../org/apache/nifi/remote/RemoteGroupPort.java |   35 -
 .../org/apache/nifi/remote/RootGroupPort.java   |   78 -
 .../apache/nifi/remote/TransferDirection.java   |   23 -
 .../nifi/remote/VersionedRemoteResource.java    |   24 -
 .../apache/nifi/remote/codec/FlowFileCodec.java |   79 -
 .../remote/exception/BadRequestException.java   |   30 -
 .../remote/exception/HandshakeException.java    |   30 -
 .../exception/NotAuthorizedException.java       |   26 -
 .../exception/PortNotRunningException.java      |   26 -
 .../remote/exception/ProtocolException.java     |   34 -
 .../exception/RequestExpiredException.java      |   26 -
 .../remote/exception/UnknownPortException.java  |   26 -
 .../nifi/remote/protocol/ClientProtocol.java    |   78 -
 .../remote/protocol/CommunicationsInput.java    |   27 -
 .../remote/protocol/CommunicationsOutput.java   |   27 -
 .../remote/protocol/CommunicationsSession.java  |   64 -
 .../nifi/remote/protocol/RequestType.java       |   43 -
 .../nifi/remote/protocol/ServerProtocol.java    |  143 -
 .../framework-bundle/framework/core/.gitignore  |    1 -
 .../framework-bundle/framework/core/pom.xml     |  121 -
 .../apache/nifi/cluster/BulletinsPayload.java   |   95 -
 .../nifi/cluster/ConnectionException.java       |   42 -
 .../nifi/cluster/DisconnectionException.java    |   42 -
 .../apache/nifi/cluster/HeartbeatPayload.java   |  170 -
 .../org/apache/nifi/connectable/LocalPort.java  |  172 -
 .../nifi/connectable/StandardConnection.java    |  336 --
 .../nifi/controller/EventDrivenWorkerQueue.java |  329 --
 .../nifi/controller/FileSystemSwapManager.java  |  768 ----
 .../apache/nifi/controller/FlowController.java  | 3579 -----------------
 .../nifi/controller/FlowFromDOMFactory.java     |  418 --
 .../controller/FlowSerializationException.java  |   48 -
 .../apache/nifi/controller/FlowSerializer.java  |   42 -
 .../FlowSynchronizationException.java           |   47 -
 .../nifi/controller/FlowSynchronizer.java       |   53 -
 .../nifi/controller/FlowUnmarshaller.java       |   78 -
 .../apache/nifi/controller/SnippetManager.java  |   96 -
 .../apache/nifi/controller/StandardCounter.java |  108 -
 .../nifi/controller/StandardFlowSerializer.java |  404 --
 .../nifi/controller/StandardFlowService.java    |  875 -----
 .../controller/StandardFlowSynchronizer.java    | 1026 -----
 .../nifi/controller/StandardProcessorNode.java  | 1243 ------
 .../apache/nifi/controller/StandardSnippet.java |  186 -
 .../org/apache/nifi/controller/Template.java    |   37 -
 .../apache/nifi/controller/TemplateManager.java |  507 ---
 .../controller/UninheritableFlowException.java  |   48 -
 .../exception/FlowFileConsumptionException.java |   38 -
 .../ReportingTaskInstantiationException.java    |   31 -
 .../exception/ValidationException.java          |   47 -
 .../nifi/controller/label/StandardLabel.java    |  110 -
 .../reporting/AbstractReportingTaskNode.java    |  111 -
 .../reporting/StandardReportingContext.java     |  132 -
 .../StandardReportingInitializationContext.java |   96 -
 .../reporting/StandardReportingTaskNode.java    |   40 -
 .../repository/BatchingSessionFactory.java      |  247 --
 .../repository/ConnectionSwapInfo.java          |   58 -
 .../repository/FileSystemRepository.java        | 1351 -------
 .../controller/repository/ProcessContext.java   |  291 --
 .../repository/ProvenanceEventEnricher.java     |   34 -
 .../repository/RepositoryPurgeException.java    |   59 -
 .../repository/RingBufferEventRepository.java   |  312 --
 .../repository/StandardCounterRepository.java   |  109 -
 .../repository/StandardFlowFileEvent.java       |  237 --
 .../repository/StandardFlowFileRecord.java      |  312 --
 .../repository/StandardProcessSession.java      | 2689 -------------
 .../StandardProcessSessionFactory.java          |   33 -
 .../repository/StandardProvenanceReporter.java  |  431 ---
 .../repository/StandardRepositoryRecord.java    |  196 -
 .../StandardRepositoryStatusReport.java         |   88 -
 .../repository/VolatileContentRepository.java   |  647 ----
 .../repository/VolatileFlowFileRepository.java  |  109 -
 .../WriteAheadFlowFileRepository.java           |  864 -----
 .../repository/claim/ContentDirection.java      |   35 -
 .../repository/claim/StandardContentClaim.java  |  154 -
 .../claim/StandardContentClaimManager.java      |  155 -
 .../repository/io/ArrayManagedOutputStream.java |  245 --
 .../repository/io/ByteCountingInputStream.java  |  100 -
 .../repository/io/ByteCountingOutputStream.java |   64 -
 .../io/DisableOnCloseOutputStream.java          |   73 -
 .../io/FlowFileAccessInputStream.java           |  168 -
 .../io/FlowFileAccessOutputStream.java          |   95 -
 .../repository/io/LimitedInputStream.java       |  107 -
 .../controller/repository/io/LongHolder.java    |   48 -
 .../controller/repository/io/MemoryManager.java |   55 -
 .../repository/io/SyncOnCloseOutputStream.java  |   58 -
 .../scheduling/ConnectableProcessContext.java   |  173 -
 .../scheduling/EventDrivenSchedulingAgent.java  |  346 --
 .../scheduling/ProcessContextFactory.java       |   51 -
 .../scheduling/QuartzSchedulingAgent.java       |  220 --
 .../controller/scheduling/ScheduleState.java    |   90 -
 .../controller/scheduling/SchedulingAgent.java  |   45 -
 .../scheduling/StandardProcessScheduler.java    |  569 ---
 .../scheduling/TimerDrivenSchedulingAgent.java  |  139 -
 .../service/ControllerServiceLoader.java        |  156 -
 .../service/StandardConfigurationContext.java   |   64 -
 ...dControllerServiceInitializationContext.java |   64 -
 .../service/StandardControllerServiceNode.java  |  125 -
 .../StandardControllerServiceProvider.java      |  196 -
 .../StandardControllerServiceReference.java     |  100 -
 .../history/StandardMetricDescriptor.java       |  128 -
 .../status/history/StandardStatusHistory.java   |   54 -
 .../status/history/StandardStatusSnapshot.java  |   70 -
 .../status/history/StatusHistoryUtil.java       |  107 -
 .../VolatileComponentStatusRepository.java      |  649 ----
 .../tasks/ContinuallyRunConnectableTask.java    |   97 -
 .../tasks/ContinuallyRunProcessorTask.java      |  185 -
 .../nifi/controller/tasks/ExpireFlowFiles.java  |  114 -
 .../controller/tasks/ReportingTaskWrapper.java  |   63 -
 .../nifi/diagnostics/DiagnosticUtils.java       |   28 -
 .../nifi/diagnostics/GarbageCollection.java     |   60 -
 .../apache/nifi/diagnostics/StorageUsage.java   |   72 -
 .../nifi/diagnostics/SystemDiagnostics.java     |  212 -
 .../diagnostics/SystemDiagnosticsFactory.java   |  133 -
 .../nifi/encrypt/EncryptionException.java       |   43 -
 .../apache/nifi/encrypt/StringEncryptor.java    |  152 -
 .../java/org/apache/nifi/engine/FlowEngine.java |  135 -
 .../events/NodeBulletinProcessingStrategy.java  |   66 -
 .../nifi/events/VolatileBulletinRepository.java |  240 --
 .../events/network/CommunicationsFailure.java   |   36 -
 .../nifi/events/network/NetworkTransfer.java    |   48 -
 .../nifi/fingerprint/FingerprintException.java  |   41 -
 .../nifi/fingerprint/FingerprintFactory.java    |  985 -----
 .../nifi/groups/StandardProcessGroup.java       | 2019 ----------
 .../org/apache/nifi/jaxb/AdaptedBulletin.java   |  100 -
 .../org/apache/nifi/jaxb/AdaptedCounter.java    |   64 -
 .../org/apache/nifi/jaxb/BulletinAdapter.java   |   59 -
 .../org/apache/nifi/jaxb/CounterAdapter.java    |   51 -
 .../org/apache/nifi/lifecycle/LifeCycle.java    |   54 -
 .../nifi/lifecycle/LifeCycleException.java      |   48 -
 .../nifi/lifecycle/LifeCycleStartException.java |   49 -
 .../nifi/lifecycle/LifeCycleStopException.java  |   48 -
 .../nifi/logging/ProcessorLogObserver.java      |   48 -
 .../repository/StandardLogRepository.java       |  173 -
 .../nifi/persistence/FlowConfigurationDAO.java  |  135 -
 .../StandardSnippetDeserializer.java            |   42 -
 .../persistence/StandardSnippetSerializer.java  |   47 -
 .../StandardXMLFlowConfigurationDAO.java        |  345 --
 .../nifi/persistence/TemplateDeserializer.java  |   42 -
 .../nifi/persistence/TemplateSerializer.java    |   47 -
 .../nifi/processor/SimpleProcessLogger.java     |  307 --
 .../nifi/processor/StandardProcessContext.java  |  145 -
 .../StandardProcessorInitializationContext.java |   49 -
 .../nifi/processor/StandardPropertyValue.java   |  163 -
 .../processor/StandardSchedulingContext.java    |  107 -
 .../processor/StandardValidationContext.java    |   97 -
 .../StandardValidationContextFactory.java       |   39 -
 .../nifi/remote/StandardRemoteProcessGroup.java | 1621 --------
 ...tandardRemoteProcessGroupPortDescriptor.java |  148 -
 .../org/apache/nifi/services/FlowService.java   |  139 -
 .../nifi/spring/FlowControllerFactoryBean.java  |  103 -
 .../spring/RingBufferEventRepositoryBean.java   |   45 -
 .../spring/StandardFlowServiceFactoryBean.java  |   98 -
 .../apache/nifi/util/ComponentStatusReport.java |  139 -
 .../java/org/apache/nifi/util/Connectables.java |   56 -
 .../java/org/apache/nifi/util/DomUtils.java     |   79 -
 .../org/apache/nifi/util/ReflectionUtils.java   |  157 -
 .../java/org/apache/nifi/util/SnippetUtils.java |  241 --
 .../ControllerServiceConfiguration.xsd          |   61 -
 .../src/main/resources/FlowConfiguration.xsd    |  335 --
 .../resources/ReportingTaskConfiguration.xsd    |   87 -
 .../core/src/main/resources/nifi-context.xml    |   49 -
 .../nifi/cluster/HeartbeatPayloadTest.java      |  121 -
 .../controller/StandardFlowServiceTest.java     |  300 --
 .../controller/TestFileSystemSwapManager.java   |  104 -
 .../repository/TestFileSystemRepository.java    |  352 --
 .../TestRingBufferEventRepository.java          |  138 -
 .../repository/TestStandardProcessSession.java  | 1177 ------
 .../TestStandardProvenanceReporter.java         |   65 -
 .../TestVolatileContentRepository.java          |  183 -
 .../TestWriteAheadFlowFileRepository.java       |  135 -
 .../repository/io/TestLimitedOutputStream.java  |   76 -
 .../controller/repository/util/DiskUtils.java   |   77 -
 .../fingerprint/FingerprintFactoryTest.java     |   60 -
 .../processor/TestStandardPropertyValue.java    |  167 -
 .../processors/DataGeneratorTestProcessor.java  |   85 -
 .../StubAttributeLoggerProcessor.java           |  111 -
 .../StubTerminationFileProcessor.java           |   40 -
 .../core/src/test/resources/conf/0bytes.xml     |    0
 .../test/resources/conf/all-flow-corrupt.xml    |  201 -
 .../resources/conf/all-flow-inheritable.xml     |  196 -
 .../resources/conf/all-flow-uninheritable.xml   |  202 -
 .../core/src/test/resources/conf/all-flow.xml   |  198 -
 .../src/test/resources/conf/nifi.properties     |  127 -
 .../test/resources/conf/only-termination.xml    |   37 -
 .../src/test/resources/conf/remote-flow.xml     |  145 -
 .../src/test/resources/conf/standard-flow.xml   |  196 -
 .../core/src/test/resources/conf/taskConfig.xml |   17 -
 .../test/resources/conf/termination-only.xml    |    0
 .../framework/core/src/test/resources/hello.txt |    1 -
 .../core/src/test/resources/logback-test.xml    |   36 -
 .../test/resources/nifi-with-remote.properties  |  127 -
 .../core/src/test/resources/nifi.properties     |  127 -
 .../test/resources/nifi/fingerprint/flow1a.xml  |  160 -
 .../test/resources/nifi/fingerprint/flow1b.xml  |  160 -
 .../test/resources/nifi/fingerprint/flow2.xml   |  160 -
 .../core/src/test/resources/old-swap-file.swap  |  Bin 1730054 -> 0 bytes
 .../file-authorization-provider/pom.xml         |   81 -
 .../FileAuthorizationProvider.java              |  584 ---
 ....apache.nifi.authorization.AuthorityProvider |   15 -
 .../src/main/xsd/users.xsd                      |   64 -
 .../FileAuthorizationProviderTest.java          |  127 -
 .../framework-bundle/framework/nar/.gitignore   |    1 -
 .../framework-bundle/framework/nar/pom.xml      |   36 -
 .../org/apache/nifi/nar/ExtensionManager.java   |  188 -
 .../org/apache/nifi/nar/ExtensionMapping.java   |   73 -
 .../org/apache/nifi/nar/NarClassLoader.java     |  227 --
 .../org/apache/nifi/nar/NarClassLoaders.java    |  301 --
 .../java/org/apache/nifi/nar/NarUnpacker.java   |  351 --
 .../java/org/apache/nifi/util/FileUtils.java    |  187 -
 nar-bundles/framework-bundle/framework/pom.xml  |   51 -
 .../framework/resources/pom.xml                 |   52 -
 .../src/main/assembly/dependencies.xml          |   36 -
 .../src/main/resources/bin/dump-nifi.bat        |   33 -
 .../src/main/resources/bin/nifi-status.bat      |   33 -
 .../resources/src/main/resources/bin/nifi.sh    |  181 -
 .../src/main/resources/bin/run-nifi.bat         |   33 -
 .../src/main/resources/bin/start-nifi.bat       |   33 -
 .../src/main/resources/bin/stop-nifi.bat        |   33 -
 .../main/resources/conf/authority-providers.xml |   43 -
 .../main/resources/conf/authorized-users.xml    |   57 -
 .../src/main/resources/conf/bootstrap.conf      |   57 -
 .../main/resources/conf/controller-services.xml |   18 -
 .../src/main/resources/conf/logback.xml         |  105 -
 .../src/main/resources/conf/nifi.properties     |  159 -
 .../src/main/resources/conf/reporting-tasks.xml |   49 -
 .../framework/runtime/.gitignore                |    2 -
 .../framework-bundle/framework/runtime/pom.xml  |   42 -
 .../java/org/apache/nifi/BootstrapListener.java |  378 --
 .../src/main/java/org/apache/nifi/NiFi.java     |  223 --
 .../main/java/org/apache/nifi/NiFiServer.java   |   31 -
 .../apache/nifi/util/LimitingInputStream.java   |  107 -
 .../framework/security/.gitignore               |    1 -
 .../framework-bundle/framework/security/pom.xml |   42 -
 .../util/SslContextCreationException.java       |   40 -
 .../security/util/SslContextFactory.java        |  119 -
 .../framework/security/util/SslException.java   |   40 -
 .../security/util/SslServerSocketFactory.java   |   81 -
 ...SslServerSocketFactoryCreationException.java |   40 -
 .../security/util/SslSocketFactory.java         |   91 -
 .../util/SslSocketFactoryCreationException.java |   40 -
 .../security/util/SslContextFactoryTest.java    |   69 -
 .../src/test/resources/localhost-ks.jks         |  Bin 1298 -> 0 bytes
 .../src/test/resources/localhost-ts.jks         |  Bin 891 -> 0 bytes
 .../src/test/resources/log4j.properties         |   21 -
 .../framework/site-to-site/.gitignore           |    1 -
 .../framework/site-to-site/pom.xml              |   74 -
 .../util/RemoteProcessGroupUtils.java           |  216 --
 .../remote/AbstractCommunicationsSession.java   |   54 -
 .../nifi/remote/RemoteResourceFactory.java      |  202 -
 .../nifi/remote/RemoteResourceManager.java      |  227 --
 .../apache/nifi/remote/RemoteSiteListener.java  |   33 -
 .../nifi/remote/SocketRemoteSiteListener.java   |  340 --
 .../nifi/remote/StandardRemoteGroupPort.java    |  764 ----
 .../nifi/remote/StandardRootGroupPort.java      |  629 ---
 .../remote/codec/StandardFlowFileCodec.java     |  169 -
 .../exception/UnsupportedCodecException.java    |   31 -
 .../SocketChannelCommunicationsSession.java     |   90 -
 .../remote/io/socket/SocketChannelInput.java    |   66 -
 .../remote/io/socket/SocketChannelOutput.java   |   58 -
 .../SSLSocketChannelCommunicationsSession.java  |   93 -
 .../io/socket/ssl/SSLSocketChannelInput.java    |   50 -
 .../io/socket/ssl/SSLSocketChannelOutput.java   |   44 -
 .../socket/ClusterManagerServerProtocol.java    |  208 -
 .../protocol/socket/HandshakeProperty.java      |   23 -
 .../nifi/remote/protocol/socket/Response.java   |   51 -
 .../remote/protocol/socket/ResponseCode.java    |  152 -
 .../protocol/socket/SocketClientProtocol.java   |  510 ---
 .../socket/SocketFlowFileServerProtocol.java    |  581 ---
 .../org.apache.nifi.remote.codec.FlowFileCodec  |   15 -
 ...g.apache.nifi.remote.protocol.ClientProtocol |   15 -
 ...g.apache.nifi.remote.protocol.ServerProtocol |   16 -
 .../remote/TestStandardRemoteGroupPort.java     |   97 -
 .../remote/TestStandardSiteToSiteProtocol.java  |  116 -
 .../io/socket/TestSocketChannelStreams.java     |  231 --
 .../io/socket/ssl/TestSSLSocketChannel.java     |  381 --
 .../test/resources/dummy-certs/localhost-ks.jks |  Bin 1298 -> 0 bytes
 .../test/resources/dummy-certs/localhost-ts.jks |  Bin 891 -> 0 bytes
 .../src/test/resources/logback-test.xml         |   30 -
 .../src/test/resources/nifi.properties          |  111 -
 .../framework/user-actions/.gitignore           |    1 -
 .../framework/user-actions/pom.xml              |   27 -
 .../java/org/apache/nifi/action/Action.java     |  121 -
 .../java/org/apache/nifi/action/Component.java  |   31 -
 .../java/org/apache/nifi/action/Operation.java  |   36 -
 .../component/details/ComponentDetails.java     |   26 -
 .../component/details/ProcessorDetails.java     |   34 -
 .../details/RemoteProcessGroupDetails.java      |   34 -
 .../nifi/action/details/ActionDetails.java      |   26 -
 .../nifi/action/details/ConfigureDetails.java   |   52 -
 .../nifi/action/details/ConnectDetails.java     |   90 -
 .../apache/nifi/action/details/MoveDetails.java |   61 -
 .../nifi/action/details/PurgeDetails.java       |   41 -
 .../web/custom-ui-utilities/.gitignore          |    1 -
 .../framework/web/custom-ui-utilities/pom.xml   |   36 -
 .../web/HttpServletRequestContextConfig.java    |  118 -
 .../framework/web/nifi-jetty/.gitignore         |    1 -
 .../framework/web/nifi-jetty/pom.xml            |  166 -
 .../src/main/assembly/distribution.xml          |   42 -
 .../org/apache/nifi/web/server/JettyServer.java |  686 ----
 .../server/ServerConfigurationException.java    |   36 -
 .../org/apache/nifi/web/webdefault.xml          |  556 ---
 .../framework/web/nifi-web-api/.gitignore       |    1 -
 .../framework/web/nifi-web-api/pom.xml          |  305 --
 .../src/main/enunciate/enunciate.xml            |   36 -
 .../src/main/enunciate/images/home.png          |  Bin 144 -> 0 bytes
 .../src/main/enunciate/override.css             |  178 -
 .../org/apache/nifi/aop/MethodProfiler.java     |   45 -
 .../apache/nifi/audit/ControllerAuditor.java    |  257 --
 .../org/apache/nifi/audit/FunnelAuditor.java    |  135 -
 .../java/org/apache/nifi/audit/NiFiAuditor.java |  119 -
 .../java/org/apache/nifi/audit/PortAuditor.java |  334 --
 .../apache/nifi/audit/ProcessGroupAuditor.java  |  263 --
 .../org/apache/nifi/audit/ProcessorAuditor.java |  408 --
 .../apache/nifi/audit/RelationshipAuditor.java  |  412 --
 .../nifi/audit/RemoteProcessGroupAuditor.java   |  376 --
 .../org/apache/nifi/audit/SnippetAuditor.java   |  519 ---
 .../nifi/web/ClusterRequestException.java       |   38 -
 .../IllegalClusterResourceRequestException.java |   33 -
 .../org/apache/nifi/web/NiFiCoreException.java  |   32 -
 .../org/apache/nifi/web/NiFiServiceFacade.java  | 1263 ------
 .../apache/nifi/web/NiFiServiceFacadeLock.java  |  177 -
 .../nifi/web/StandardNiFiServiceFacade.java     | 2877 --------------
 .../apache/nifi/web/StandardNiFiWebContext.java |  336 --
 .../nifi/web/api/ApplicationResource.java       |  399 --
 .../nifi/web/api/BulletinBoardResource.java     |  116 -
 .../apache/nifi/web/api/ClusterResource.java    |  724 ----
 .../apache/nifi/web/api/ConnectionResource.java |  771 ----
 .../apache/nifi/web/api/ControllerResource.java |  803 ----
 .../org/apache/nifi/web/api/FunnelResource.java |  493 ---
 .../apache/nifi/web/api/HistoryResource.java    |  271 --
 .../apache/nifi/web/api/InputPortResource.java  |  531 ---
 .../org/apache/nifi/web/api/LabelResource.java  |  547 ---
 .../org/apache/nifi/web/api/NodeResource.java   |  302 --
 .../apache/nifi/web/api/OutputPortResource.java |  532 ---
 .../nifi/web/api/ProcessGroupResource.java      | 1102 ------
 .../apache/nifi/web/api/ProcessorResource.java  |  693 ----
 .../apache/nifi/web/api/ProvenanceResource.java | 1068 ------
 .../web/api/RemoteProcessGroupResource.java     |  912 -----
 .../apache/nifi/web/api/SnippetResource.java    |  601 ---
 .../nifi/web/api/SystemDiagnosticsResource.java |   81 -
 .../apache/nifi/web/api/TemplateResource.java   |  400 --
 .../apache/nifi/web/api/UserGroupResource.java  |  383 --
 .../org/apache/nifi/web/api/UserResource.java   |  469 ---
 .../api/config/AccessDeniedExceptionMapper.java |   54 -
 .../config/AccountNotFoundExceptionMapper.java  |   47 -
 .../config/AdministrationExceptionMapper.java   |   48 -
 ...ationCredentialsNotFoundExceptionMapper.java |   48 -
 .../web/api/config/ClusterExceptionMapper.java  |   47 -
 ...ectingNodeMutableRequestExceptionMapper.java |   49 -
 ...nectedNodeMutableRequestExceptionMapper.java |   49 -
 .../config/IllegalArgumentExceptionMapper.java  |   46 -
 ...alClusterResourceRequestExceptionMapper.java |   47 -
 .../IllegalClusterStateExceptionMapper.java     |   47 -
 .../IllegalNodeDeletionExceptionMapper.java     |   46 -
 ...IllegalNodeDisconnectionExceptionMapper.java |   46 -
 .../IllegalNodeReconnectionExceptionMapper.java |   46 -
 .../api/config/IllegalStateExceptionMapper.java |   46 -
 .../IneligiblePrimaryNodeExceptionMapper.java   |   47 -
 .../config/InvalidRevisionExceptionMapper.java  |   47 -
 .../api/config/JsonMappingExceptionMapper.java  |   46 -
 .../api/config/JsonParseExceptionMapper.java    |   46 -
 .../config/MutableRequestExceptionMapper.java   |   48 -
 .../web/api/config/NiFiCoreExceptionMapper.java |   47 -
 .../config/NoConnectedNodesExceptionMapper.java |   47 -
 .../NoResponseFromNodesExceptionMapper.java     |   49 -
 .../NodeDisconnectionExceptionMapper.java       |   47 -
 .../config/NodeReconnectionExceptionMapper.java |   47 -
 .../PrimaryRoleAssignmentExceptionMapper.java   |   47 -
 .../config/ResourceNotFoundExceptionMapper.java |   48 -
 .../SafeModeMutableRequestExceptionMapper.java  |   48 -
 .../nifi/web/api/config/ThrowableMapper.java    |   46 -
 .../api/config/UnknownNodeExceptionMapper.java  |   47 -
 .../api/config/ValidationExceptionMapper.java   |   49 -
 .../config/WebApplicationExceptionMapper.java   |   63 -
 .../org/apache/nifi/web/api/dto/DtoFactory.java | 2142 -----------
 .../org/apache/nifi/web/api/package-info.java   |   58 -
 .../request/BulletinBoardPatternParameter.java  |   47 -
 .../nifi/web/api/request/ClientIdParameter.java |   45 -
 .../api/request/ConnectableTypeParameter.java   |   41 -
 .../nifi/web/api/request/DateTimeParameter.java |   63 -
 .../nifi/web/api/request/DoubleParameter.java   |   40 -
 .../nifi/web/api/request/IntegerParameter.java  |   40 -
 .../nifi/web/api/request/LongParameter.java     |   40 -
 .../ApplicationStartupContextListener.java      |  157 -
 .../nifi/web/controller/ControllerFacade.java   | 1392 -------
 .../web/controller/StandardSearchContext.java   |   64 -
 .../org/apache/nifi/web/dao/ConnectionDAO.java  |  112 -
 .../java/org/apache/nifi/web/dao/FunnelDAO.java |   88 -
 .../java/org/apache/nifi/web/dao/LabelDAO.java  |   79 -
 .../java/org/apache/nifi/web/dao/PortDAO.java   |   96 -
 .../apache/nifi/web/dao/ProcessGroupDAO.java    |   89 -
 .../org/apache/nifi/web/dao/ProcessorDAO.java   |   97 -
 .../nifi/web/dao/RemoteProcessGroupDAO.java     |  135 -
 .../org/apache/nifi/web/dao/SnippetDAO.java     |   92 -
 .../org/apache/nifi/web/dao/TemplateDAO.java    |   77 -
 .../apache/nifi/web/dao/impl/ComponentDAO.java  |   72 -
 .../web/dao/impl/StandardConnectionDAO.java     |  519 ---
 .../nifi/web/dao/impl/StandardFunnelDAO.java    |  180 -
 .../nifi/web/dao/impl/StandardInputPortDAO.java |  309 --
 .../nifi/web/dao/impl/StandardLabelDAO.java     |  184 -
 .../web/dao/impl/StandardOutputPortDAO.java     |  309 --
 .../web/dao/impl/StandardProcessGroupDAO.java   |  171 -
 .../nifi/web/dao/impl/StandardProcessorDAO.java |  503 ---
 .../dao/impl/StandardRemoteProcessGroupDAO.java |  402 --
 .../nifi/web/dao/impl/StandardSnippetDAO.java   |  344 --
 .../nifi/web/dao/impl/StandardTemplateDAO.java  |  141 -
 .../nifi/web/filter/NodeRequestFilter.java      |  130 -
 .../apache/nifi/web/filter/RequestLogger.java   |   78 -
 .../nifi/web/filter/ThreadLocalFilter.java      |   54 -
 .../org/apache/nifi/web/filter/TimerFilter.java |   70 -
 .../nifi/web/util/ClientResponseUtils.java      |   48 -
 .../nifi/web/util/DownloadableContent.java      |   47 -
 .../org/apache/nifi/web/util/SnippetUtils.java  |  426 ---
 .../src/main/resources/nifi-web-api-context.xml |  303 --
 .../src/main/webapp/WEB-INF/web.xml             |  105 -
 .../apache/nifi/integration/NiFiWebApiTest.java |  274 --
 .../accesscontrol/AdminAccessControlTest.java   | 1090 ------
 .../accesscontrol/DfmAccessControlTest.java     | 1410 -------
 .../ReadOnlyAccessControlTest.java              | 1064 -----
 .../util/NiFiTestAuthorizationProvider.java     |  179 -
 .../nifi/integration/util/NiFiTestServer.java   |  235 --
 .../nifi/integration/util/NiFiTestUser.java     |  257 --
 .../integration/util/SourceTestProcessor.java   |   49 -
 .../util/TerminationTestProcessor.java          |   44 -
 .../nifi/integration/util/TestPrioritizer.java  |   31 -
 ....apache.nifi.authorization.AuthorityProvider |   15 -
 ...org.apache.nifi.flowfile.FlowFilePrioritizer |   15 -
 .../org.apache.nifi.processor.Processor         |   16 -
 .../access-control/authority-providers.xml      |   24 -
 .../access-control/controller-services.xml      |   18 -
 .../resources/access-control/localhost-ks.jks   |  Bin 2246 -> 0 bytes
 .../resources/access-control/localhost-ts.jks   |  Bin 957 -> 0 bytes
 .../resources/access-control/nifi.properties    |  136 -
 .../access-control/reporting-tasks.xml          |   17 -
 .../src/test/resources/logback-test.xml         |   36 -
 .../framework/web/nifi-web-docs/.gitignore      |    1 -
 .../framework/web/nifi-web-docs/pom.xml         |   57 -
 .../nifi/web/docs/DocumentationController.java  |   91 -
 .../main/webapp/WEB-INF/jsp/documentation.jsp   |  130 -
 .../WEB-INF/jsp/no-documentation-found.jsp      |   31 -
 .../src/main/webapp/WEB-INF/web.xml             |   33 -
 .../src/main/webapp/css/component-usage.css     |  127 -
 .../nifi-web-docs/src/main/webapp/css/main.css  |  214 --
 .../src/main/webapp/images/bgBannerFoot.png     |  Bin 189 -> 0 bytes
 .../src/main/webapp/images/bgHeader.png         |  Bin 1697 -> 0 bytes
 .../src/main/webapp/images/bgTableHeader.png    |  Bin 232 -> 0 bytes
 .../src/main/webapp/js/application.js           |  282 --
 .../framework/web/nifi-web-error/.gitignore     |    1 -
 .../framework/web/nifi-web-error/pom.xml        |   48 -
 .../apache/nifi/web/filter/CatchAllFilter.java  |   45 -
 .../src/main/webapp/WEB-INF/web.xml             |   26 -
 .../nifi-web-error/src/main/webapp/index.jsp    |   37 -
 .../framework/web/nifi-web-ui/.gitignore        |    1 -
 .../framework/web/nifi-web-ui/pom.xml           |  648 ----
 .../apache/nifi/web/filter/IeEdgeHeader.java    |   57 -
 .../org/apache/nifi/web/servlet/ConvertSvg.java |  117 -
 .../filters/bulletin-board-min.properties       |   18 -
 .../resources/filters/bulletin-board.properties |   25 -
 .../resources/filters/canvas-min.properties     |   18 -
 .../main/resources/filters/canvas.properties    |   69 -
 .../resources/filters/cluster-min.properties    |   18 -
 .../main/resources/filters/cluster.properties   |   26 -
 .../resources/filters/counters-min.properties   |   18 -
 .../main/resources/filters/counters.properties  |   26 -
 .../resources/filters/history-min.properties    |   18 -
 .../main/resources/filters/history.properties   |   27 -
 .../resources/filters/provenance-min.properties |   18 -
 .../resources/filters/provenance.properties     |   26 -
 .../resources/filters/summary-min.properties    |   18 -
 .../main/resources/filters/summary.properties   |   32 -
 .../resources/filters/templates-min.properties  |   18 -
 .../main/resources/filters/templates.properties |   26 -
 .../main/resources/filters/users-min.properties |   18 -
 .../src/main/resources/filters/users.properties |   26 -
 .../webapp/WEB-INF/pages/bulletin-board.jsp     |   50 -
 .../src/main/webapp/WEB-INF/pages/canvas.jsp    |  118 -
 .../src/main/webapp/WEB-INF/pages/cluster.jsp   |   62 -
 .../src/main/webapp/WEB-INF/pages/counters.jsp  |   61 -
 .../src/main/webapp/WEB-INF/pages/history.jsp   |   65 -
 .../main/webapp/WEB-INF/pages/message-page.jsp  |   34 -
 .../main/webapp/WEB-INF/pages/provenance.jsp    |   68 -
 .../src/main/webapp/WEB-INF/pages/summary.jsp   |   80 -
 .../src/main/webapp/WEB-INF/pages/templates.jsp |   63 -
 .../src/main/webapp/WEB-INF/pages/users.jsp     |   71 -
 .../webapp/WEB-INF/partials/banners-main.jsp    |   19 -
 .../webapp/WEB-INF/partials/banners-utility.jsp |   19 -
 .../bulletin-board/bulletin-board-content.jsp   |   40 -
 .../WEB-INF/partials/canvas/about-dialog.jsp    |   29 -
 .../WEB-INF/partials/canvas/canvas-header.jsp   |   56 -
 .../canvas/connection-configuration.jsp         |  180 -
 .../partials/canvas/connections-dialog.jsp      |   35 -
 .../partials/canvas/fill-color-dialog.jsp       |   42 -
 .../WEB-INF/partials/canvas/flow-status.jsp     |   55 -
 .../canvas/instantiate-template-dialog.jsp      |   22 -
 .../partials/canvas/label-configuration.jsp     |   38 -
 .../WEB-INF/partials/canvas/navigation.jsp      |   57 -
 .../WEB-INF/partials/canvas/new-port-dialog.jsp |   27 -
 .../canvas/new-process-group-dialog.jsp         |   27 -
 .../partials/canvas/new-processor-dialog.jsp    |   50 -
 .../canvas/new-processor-property-dialog.jsp    |   34 -
 .../canvas/new-remote-process-group-dialog.jsp  |   27 -
 .../partials/canvas/new-template-dialog.jsp     |   33 -
 .../partials/canvas/port-configuration.jsp      |   54 -
 .../WEB-INF/partials/canvas/port-details.jsp    |   39 -
 .../canvas/process-group-configuration.jsp      |   34 -
 .../partials/canvas/process-group-details.jsp   |   33 -
 .../partials/canvas/processor-configuration.jsp |  213 --
 .../WEB-INF/partials/canvas/registration.jsp    |   44 -
 .../canvas/remote-port-configuration.jsp        |   42 -
 .../remote-process-group-configuration.jsp      |   50 -
 .../canvas/remote-process-group-details.jsp     |   48 -
 .../canvas/remote-process-group-ports.jsp       |   47 -
 .../canvas/secure-port-configuration.jsp        |   82 -
 .../partials/canvas/secure-port-details.jsp     |   67 -
 .../partials/canvas/settings-content.jsp        |   60 -
 .../webapp/WEB-INF/partials/canvas/shell.jsp    |   27 -
 .../partials/cluster/cluster-content.jsp        |   39 -
 .../partials/cluster/node-details-dialog.jsp    |   42 -
 .../WEB-INF/partials/connection-details.jsp     |  123 -
 .../partials/counters/counters-content.jsp      |   39 -
 .../partials/history/action-details-dialog.jsp  |   27 -
 .../partials/history/history-content.jsp        |   36 -
 .../partials/history/history-filter-dialog.jsp  |   73 -
 .../partials/history/history-purge-dialog.jsp   |   42 -
 .../webapp/WEB-INF/partials/message-pane.jsp    |   23 -
 .../main/webapp/WEB-INF/partials/ok-dialog.jsp  |   22 -
 .../WEB-INF/partials/processor-details.jsp      |  158 -
 .../provenance/lineage-query-dialog.jsp         |   26 -
 .../partials/provenance/provenance-content.jsp  |   73 -
 .../provenance-event-details-dialog.jsp         |  196 -
 .../provenance/provenance-query-dialog.jsp      |   26 -
 .../provenance/provenance-search-dialog.jsp     |  101 -
 .../WEB-INF/partials/status-history-dialog.jsp  |   37 -
 .../cluster-connection-summary-dialog.jsp       |   36 -
 .../cluster-input-port-summary-dialog.jsp       |   36 -
 .../cluster-output-port-summary-dialog.jsp      |   36 -
 .../cluster-processor-summary-dialog.jsp        |   36 -
 ...ster-remote-process-group-summary-dialog.jsp |   36 -
 .../partials/summary/summary-content.jsp        |   64 -
 .../summary/system-diagnostics-dialog.jsp       |  131 -
 .../summary/view-single-node-dialog.jsp         |   24 -
 .../partials/templates/templates-content.jsp    |   59 -
 .../partials/users/group-revoke-dialog.jsp      |   22 -
 .../partials/users/group-roles-dialog.jsp       |   52 -
 .../partials/users/user-delete-dialog.jsp       |   23 -
 .../partials/users/user-details-dialog.jsp      |   56 -
 .../partials/users/user-group-dialog.jsp        |   27 -
 .../partials/users/user-revoke-dialog.jsp       |   23 -
 .../partials/users/user-roles-dialog.jsp        |   60 -
 .../WEB-INF/partials/users/users-content.jsp    |   46 -
 .../webapp/WEB-INF/partials/yes-no-dialog.jsp   |   22 -
 .../nifi-web-ui/src/main/webapp/WEB-INF/web.xml |  120 -
 .../nifi-web-ui/src/main/webapp/css/about.css   |   54 -
 .../nifi-web-ui/src/main/webapp/css/banner.css  |   80 -
 .../src/main/webapp/css/bulletin-board.css      |  238 --
 .../nifi-web-ui/src/main/webapp/css/canvas.css  |   41 -
 .../nifi-web-ui/src/main/webapp/css/cluster.css |  152 -
 .../webapp/css/connection-configuration.css     |  168 -
 .../src/main/webapp/css/connection-details.css  |   77 -
 .../src/main/webapp/css/counters.css            |  134 -
 .../nifi-web-ui/src/main/webapp/css/dialog.css  |  351 --
 .../src/main/webapp/css/flow-status.css         |  110 -
 .../nifi-web-ui/src/main/webapp/css/graph.css   |  262 --
 .../nifi-web-ui/src/main/webapp/css/header.css  |  683 ----
 .../nifi-web-ui/src/main/webapp/css/history.css |  216 --
 .../src/main/webapp/css/label-configuration.css |   55 -
 .../nifi-web-ui/src/main/webapp/css/main.css    |  298 --
 .../src/main/webapp/css/message-page.css        |   30 -
 .../src/main/webapp/css/message-pane.css        |   44 -
 .../src/main/webapp/css/navigation.css          |  292 --
 .../main/webapp/css/new-processor-dialog.css    |  168 -
 .../src/main/webapp/css/port-configuration.css  |  192 -
 .../src/main/webapp/css/port-details.css        |   55 -
 .../webapp/css/process-group-configuration.css  |   40 -
 .../main/webapp/css/process-group-details.css   |   27 -
 .../main/webapp/css/processor-configuration.css |  388 --
 .../src/main/webapp/css/processor-details.css   |  119 -
 .../src/main/webapp/css/provenance.css          |  654 ----
 .../src/main/webapp/css/registration.css        |   53 -
 .../css/remote-process-group-configuration.css  |  233 --
 .../nifi-web-ui/src/main/webapp/css/reset.css   |   43 -
 .../src/main/webapp/css/settings.css            |   66 -
 .../nifi-web-ui/src/main/webapp/css/shell.css   |   98 -
 .../src/main/webapp/css/status-history.css      |  199 -
 .../nifi-web-ui/src/main/webapp/css/summary.css |  768 ----
 .../src/main/webapp/css/templates.css           |  229 --
 .../nifi-web-ui/src/main/webapp/css/users.css   |  254 --
 .../src/main/webapp/images/addConnect.png       |  Bin 1996 -> 0 bytes
 .../src/main/webapp/images/bg-error.png         |  Bin 63766 -> 0 bytes
 .../src/main/webapp/images/bgBannerFoot.png     |  Bin 189 -> 0 bytes
 .../src/main/webapp/images/bgBannerHead.png     |  Bin 422 -> 0 bytes
 .../src/main/webapp/images/bgBreadcrumbs.png    |  Bin 2130 -> 0 bytes
 .../src/main/webapp/images/bgButton.png         |  Bin 234 -> 0 bytes
 .../src/main/webapp/images/bgButtonOver.png     |  Bin 234 -> 0 bytes
 .../src/main/webapp/images/bgButtonSelected.png |  Bin 166 -> 0 bytes
 .../main/webapp/images/bgButtonSelectedOver.png |  Bin 168 -> 0 bytes
 .../src/main/webapp/images/bgContextMenu.png    |  Bin 746 -> 0 bytes
 .../src/main/webapp/images/bgControlsInset.png  |  Bin 618 -> 0 bytes
 .../src/main/webapp/images/bgHeader.png         |  Bin 1697 -> 0 bytes
 .../src/main/webapp/images/bgInputText.png      |  Bin 139 -> 0 bytes
 .../src/main/webapp/images/bgNifiLogo.png       |  Bin 4232 -> 0 bytes
 .../src/main/webapp/images/bgNifiToolbar.png    |  Bin 2123 -> 0 bytes
 .../main/webapp/images/bgNifiToolbarEdge.png    |  Bin 541 -> 0 bytes
 .../src/main/webapp/images/bgOutline.png        |  Bin 585 -> 0 bytes
 .../src/main/webapp/images/bgPanCenter.png      |  Bin 362 -> 0 bytes
 .../src/main/webapp/images/bgPanZoom.png        |  Bin 832 -> 0 bytes
 .../webapp/images/bgProcessGroupDetailsArea.png |  Bin 6596 -> 0 bytes
 .../main/webapp/images/bgProcessorStatArea.png  |  Bin 4062 -> 0 bytes
 .../images/bgRemoteProcessGroupDetailsArea.png  |  Bin 4433 -> 0 bytes
 .../src/main/webapp/images/bgSearchInput.png    |  Bin 221 -> 0 bytes
 .../src/main/webapp/images/bgShellClose.png     |  Bin 169 -> 0 bytes
 .../src/main/webapp/images/bgStatusPanel.png    |  Bin 229 -> 0 bytes
 .../src/main/webapp/images/bgTabContainer.png   |  Bin 234 -> 0 bytes
 .../src/main/webapp/images/bgTableHeader.png    |  Bin 232 -> 0 bytes
 .../main/webapp/images/bgToolbarBtnBorder.png   |  Bin 227 -> 0 bytes
 .../src/main/webapp/images/blueBtnBg-over.jpg   |  Bin 356 -> 0 bytes
 .../src/main/webapp/images/blueBtnBg.jpg        |  Bin 356 -> 0 bytes
 .../src/main/webapp/images/buttonColor.png      |  Bin 1679 -> 0 bytes
 .../src/main/webapp/images/buttonCopy.png       |  Bin 1166 -> 0 bytes
 .../src/main/webapp/images/buttonDelete.png     |  Bin 1837 -> 0 bytes
 .../src/main/webapp/images/buttonDisable.png    |  Bin 1737 -> 0 bytes
 .../src/main/webapp/images/buttonEnable.png     |  Bin 1375 -> 0 bytes
 .../src/main/webapp/images/buttonGroup.png      |  Bin 880 -> 0 bytes
 .../main/webapp/images/buttonNewProperty.png    |  Bin 590 -> 0 bytes
 .../src/main/webapp/images/buttonOutline.png    |  Bin 495 -> 0 bytes
 .../src/main/webapp/images/buttonPanDown.png    |  Bin 554 -> 0 bytes
 .../src/main/webapp/images/buttonPanLeft.png    |  Bin 588 -> 0 bytes
 .../src/main/webapp/images/buttonPanRight.png   |  Bin 530 -> 0 bytes
 .../src/main/webapp/images/buttonPanUp.png      |  Bin 551 -> 0 bytes
 .../src/main/webapp/images/buttonPaste.png      |  Bin 1372 -> 0 bytes
 .../src/main/webapp/images/buttonRefresh.png    |  Bin 915 -> 0 bytes
 .../src/main/webapp/images/buttonRun.png        |  Bin 1163 -> 0 bytes
 .../src/main/webapp/images/buttonStop.png       |  Bin 869 -> 0 bytes
 .../src/main/webapp/images/buttonTemplate.png   |  Bin 1290 -> 0 bytes
 .../src/main/webapp/images/buttonZoom100.png    |  Bin 449 -> 0 bytes
 .../src/main/webapp/images/buttonZoomFit.png    |  Bin 499 -> 0 bytes
 .../src/main/webapp/images/buttonZoomIn.png     |  Bin 435 -> 0 bytes
 .../src/main/webapp/images/buttonZoomOut.png    |  Bin 339 -> 0 bytes
 .../src/main/webapp/images/cluster.png          |  Bin 326 -> 0 bytes
 .../src/main/webapp/images/grayBtnBg-over.jpg   |  Bin 319 -> 0 bytes
 .../src/main/webapp/images/grayBtnBg.jpg        |  Bin 317 -> 0 bytes
 .../nifi-web-ui/src/main/webapp/images/grid.gif |  Bin 135 -> 0 bytes
 .../src/main/webapp/images/headerTabBg.gif      |  Bin 131 -> 0 bytes
 .../src/main/webapp/images/iconAdminUser.png    |  Bin 1960 -> 0 bytes
 .../src/main/webapp/images/iconAlert.png        |  Bin 1396 -> 0 bytes
 .../src/main/webapp/images/iconAlertDialog.png  |  Bin 1241 -> 0 bytes
 .../src/main/webapp/images/iconAutoRefresh.png  |  Bin 3102 -> 0 bytes
 .../src/main/webapp/images/iconBulletin.png     |  Bin 1066 -> 0 bytes
 .../main/webapp/images/iconBulletinBoard.png    |  Bin 2523 -> 0 bytes
 .../src/main/webapp/images/iconCenterView.png   |  Bin 338 -> 0 bytes
 .../src/main/webapp/images/iconChart.png        |  Bin 510 -> 0 bytes
 .../src/main/webapp/images/iconClose.png        |  Bin 231 -> 0 bytes
 .../src/main/webapp/images/iconCluster.png      |  Bin 2112 -> 0 bytes
 .../src/main/webapp/images/iconClusterSmall.png |  Bin 757 -> 0 bytes
 .../main/webapp/images/iconCollapseChildren.png |  Bin 571 -> 0 bytes
 .../main/webapp/images/iconCollapseParents.png  |  Bin 645 -> 0 bytes
 .../src/main/webapp/images/iconColor.png        |  Bin 738 -> 0 bytes
 .../src/main/webapp/images/iconCommit.png       |  Bin 445 -> 0 bytes
 .../src/main/webapp/images/iconConfigure.png    |  Bin 696 -> 0 bytes
 .../src/main/webapp/images/iconConnect.png      |  Bin 589 -> 0 bytes
 .../src/main/webapp/images/iconConnection.png   |  Bin 1517 -> 0 bytes
 .../src/main/webapp/images/iconCopy.png         |  Bin 514 -> 0 bytes
 .../src/main/webapp/images/iconCounter.png      |  Bin 1369 -> 0 bytes
 .../src/main/webapp/images/iconDelete.png       |  Bin 670 -> 0 bytes
 .../src/main/webapp/images/iconDetails.png      |  Bin 549 -> 0 bytes
 .../src/main/webapp/images/iconDisable.png      |  Bin 764 -> 0 bytes
 .../src/main/webapp/images/iconDisconnect.png   |  Bin 569 -> 0 bytes
 .../src/main/webapp/images/iconEdit.png         |  Bin 493 -> 0 bytes
 .../src/main/webapp/images/iconEditButton.png   |  Bin 915 -> 0 bytes
 .../main/webapp/images/iconExpandChildren.png   |  Bin 521 -> 0 bytes
 .../main/webapp/images/iconExpandParents.png    |  Bin 585 -> 0 bytes
 .../src/main/webapp/images/iconExport.png       |  Bin 453 -> 0 bytes
 .../main/webapp/images/iconExportLineage.png    |  Bin 405 -> 0 bytes
 .../src/main/webapp/images/iconFlowSettings.png |  Bin 2624 -> 0 bytes
 .../src/main/webapp/images/iconFunnel.png       |  Bin 1223 -> 0 bytes
 .../src/main/webapp/images/iconGoTo.png         |  Bin 448 -> 0 bytes
 .../src/main/webapp/images/iconHistory.png      |  Bin 2347 -> 0 bytes
 .../src/main/webapp/images/iconInfo.png         |  Bin 550 -> 0 bytes
 .../src/main/webapp/images/iconInputPort.png    |  Bin 1842 -> 0 bytes
 .../main/webapp/images/iconInputPortSmall.png   |  Bin 532 -> 0 bytes
 .../webapp/images/iconIsolatedProcessor.png     |  Bin 1781 -> 0 bytes
 .../src/main/webapp/images/iconLineage.png      |  Bin 432 -> 0 bytes
 .../src/main/webapp/images/iconLoading.gif      |  Bin 2438 -> 0 bytes
 .../src/main/webapp/images/iconNotSecure.png    |  Bin 137 -> 0 bytes
 .../src/main/webapp/images/iconOutputPort.png   |  Bin 1658 -> 0 bytes
 .../main/webapp/images/iconOutputPortSmall.png  |  Bin 459 -> 0 bytes
 .../src/main/webapp/images/iconPaste.png        |  Bin 601 -> 0 bytes
 .../src/main/webapp/images/iconPopOut.png       |  Bin 217 -> 0 bytes
 .../webapp/images/iconPortNotTransmitting.png   |  Bin 231 -> 0 bytes
 .../src/main/webapp/images/iconPortRemoved.png  |  Bin 456 -> 0 bytes
 .../main/webapp/images/iconPortTransmitting.png |  Bin 235 -> 0 bytes
 .../src/main/webapp/images/iconPrimary.png      |  Bin 647 -> 0 bytes
 .../src/main/webapp/images/iconProcessor.png    |  Bin 1446 -> 0 bytes
 .../src/main/webapp/images/iconProvenance.png   |  Bin 1104 -> 0 bytes
 .../src/main/webapp/images/iconRefresh.png      |  Bin 492 -> 0 bytes
 .../src/main/webapp/images/iconRemotePorts.png  |  Bin 456 -> 0 bytes
 .../webapp/images/iconRemoteProcessGroup.png    |  Bin 674 -> 0 bytes
 .../src/main/webapp/images/iconReporting.png    |  Bin 803 -> 0 bytes
 .../src/main/webapp/images/iconResetCounter.png |  Bin 304 -> 0 bytes
 .../src/main/webapp/images/iconResize.png       |  Bin 142 -> 0 bytes
 .../src/main/webapp/images/iconRevoke.png       |  Bin 676 -> 0 bytes
 .../src/main/webapp/images/iconRun.png          |  Bin 538 -> 0 bytes
 .../src/main/webapp/images/iconSecure.png       |  Bin 133 -> 0 bytes
 .../src/main/webapp/images/iconSmallFunnel.png  |  Bin 455 -> 0 bytes
 .../main/webapp/images/iconSmallInputPort.png   |  Bin 216 -> 0 bytes
 .../main/webapp/images/iconSmallOutputPort.png  |  Bin 264 -> 0 bytes
 .../webapp/images/iconSmallProcessGroup.png     |  Bin 479 -> 0 bytes
 .../main/webapp/images/iconSmallProcessor.png   |  Bin 647 -> 0 bytes
 .../webapp/images/iconSmallRelationship.png     |  Bin 770 -> 0 bytes
 .../images/iconSmallRemoteProcessGroup.png      |  Bin 402 -> 0 bytes
 .../src/main/webapp/images/iconStop.png         |  Bin 402 -> 0 bytes
 .../src/main/webapp/images/iconTemplates.png    |  Bin 2659 -> 0 bytes
 .../src/main/webapp/images/iconToFront.png      |  Bin 475 -> 0 bytes
 .../webapp/images/iconTransmissionActive.png    |  Bin 1330 -> 0 bytes
 .../webapp/images/iconTransmissionInactive.png  |  Bin 1248 -> 0 bytes
 .../src/main/webapp/images/iconTwistArrow.png   |  Bin 256 -> 0 bytes
 .../src/main/webapp/images/iconUndo.png         |  Bin 642 -> 0 bytes
 .../src/main/webapp/images/iconUsage.png        |  Bin 470 -> 0 bytes
 .../src/main/webapp/images/inputCheckbox.png    |  Bin 389 -> 0 bytes
 .../src/main/webapp/images/loadAnimation.gif    |  Bin 10789 -> 0 bytes
 .../src/main/webapp/images/nifi16.ico           |  Bin 1150 -> 0 bytes
 .../src/main/webapp/images/nifi_about.png       |  Bin 16323 -> 0 bytes
 .../src/main/webapp/images/panelBg.jpg          |  Bin 312 -> 0 bytes
 .../src/main/webapp/images/portRemoved.png      |  Bin 1090 -> 0 bytes
 .../src/main/webapp/images/portRunning.png      |  Bin 337 -> 0 bytes
 .../src/main/webapp/images/portStopped.png      |  Bin 192 -> 0 bytes
 .../src/main/webapp/images/spacer.png           |  Bin 110 -> 0 bytes
 .../src/main/webapp/images/starburst.png        |  Bin 190 -> 0 bytes
 .../src/main/webapp/images/tabBg.jpg            |  Bin 320 -> 0 bytes
 .../src/main/webapp/images/toolbarBg.gif        |  Bin 45 -> 0 bytes
 .../main/webapp/images/toolboxIconFunnel.png    |  Bin 2409 -> 0 bytes
 .../main/webapp/images/toolboxIconInputPort.png |  Bin 2680 -> 0 bytes
 .../src/main/webapp/images/toolboxIconLabel.png |  Bin 2871 -> 0 bytes
 .../webapp/images/toolboxIconOutputPort.png     |  Bin 2209 -> 0 bytes
 .../webapp/images/toolboxIconProcessGroup.png   |  Bin 4176 -> 0 bytes
 .../main/webapp/images/toolboxIconProcessor.png |  Bin 4426 -> 0 bytes
 .../images/toolboxIconRemoteProcessGroup.png    |  Bin 5653 -> 0 bytes
 .../main/webapp/images/toolboxIconTemplate.png  |  Bin 2913 -> 0 bytes
 .../images/transmissionSwitchDisabled.png       |  Bin 1240 -> 0 bytes
 .../webapp/images/transmissionSwitchEnabled.png |  Bin 1600 -> 0 bytes
 .../src/main/webapp/images/ungroup.png          |  Bin 3409 -> 0 bytes
 .../src/main/webapp/js/codemirror/LICENSE       |   19 -
 .../js/codemirror/addon/hint/show-hint.css      |   38 -
 .../js/codemirror/lib/codemirror-compressed.js  |   16 -
 .../webapp/js/codemirror/lib/codemirror.css     |  318 --
 .../nifi-web-ui/src/main/webapp/js/d3/LICENSE   |   26 -
 .../nifi-web-ui/src/main/webapp/js/d3/d3.min.js |    5 -
 .../webapp/js/jquery/combo/jquery.combo.css     |   88 -
 .../main/webapp/js/jquery/combo/jquery.combo.js |  311 --
 .../main/webapp/js/jquery/combo/menuArrow.png   |  Bin 251 -> 0 bytes
 .../webapp/js/jquery/combo/menuArrowSmall.png   |  Bin 233 -> 0 bytes
 .../main/webapp/js/jquery/jquery-2.1.1.min.js   |    4 -
 .../src/main/webapp/js/jquery/jquery.center.js  |   35 -
 .../src/main/webapp/js/jquery/jquery.count.js   |   99 -
 .../src/main/webapp/js/jquery/jquery.each.js    |   31 -
 .../main/webapp/js/jquery/jquery.ellipsis.js    |  164 -
 .../js/jquery/jquery.event.drag-2.2.min.js      |    6 -
 .../main/webapp/js/jquery/jquery.form.min.js    |   11 -
 .../src/main/webapp/js/jquery/jquery.tab.js     |   69 -
 .../js/jquery/minicolors/jquery.minicolors.css  |  259 --
 .../jquery/minicolors/jquery.minicolors.min.js  |    9 -
 .../js/jquery/minicolors/jquery.minicolors.png  |  Bin 77459 -> 0 bytes
 .../webapp/js/jquery/modal/jquery.modal.css     |  110 -
 .../main/webapp/js/jquery/modal/jquery.modal.js |  266 --
 .../js/jquery/nfeditor/jquery.nfeditor.css      |   73 -
 .../js/jquery/nfeditor/jquery.nfeditor.js       |  315 --
 .../js/jquery/nfeditor/languages/nfel.css       |   51 -
 .../webapp/js/jquery/nfeditor/languages/nfel.js |  831 ----
 .../webapp/js/jquery/qtip2/jquery.qtip.min.css  |    3 -
 .../webapp/js/jquery/qtip2/jquery.qtip.min.js   |    5 -
 .../webapp/js/jquery/slickgrid/MIT-LICENSE.txt  |   20 -
 .../js/jquery/slickgrid/css/images/collapse.gif |  Bin 846 -> 0 bytes
 .../js/jquery/slickgrid/css/images/expand.gif   |  Bin 851 -> 0 bytes
 .../js/jquery/slickgrid/css/images/sort-asc.gif |  Bin 59 -> 0 bytes
 .../jquery/slickgrid/css/images/sort-desc.gif   |  Bin 59 -> 0 bytes
 .../slickgrid/css/slick-default-theme.css       |  135 -
 .../js/jquery/slickgrid/css/slick.grid.css      |  158 -
 .../slickgrid/plugins/slick.autotooltips.js     |   83 -
 .../plugins/slick.cellrangedecorator.js         |   66 -
 .../plugins/slick.cellrangeselector.js          |  113 -
 .../plugins/slick.cellselectionmodel.js         |  154 -
 .../plugins/slick.rowselectionmodel.js          |  187 -
 .../webapp/js/jquery/slickgrid/slick.core.js    |  467 ---
 .../js/jquery/slickgrid/slick.dataview.js       | 1126 ------
 .../webapp/js/jquery/slickgrid/slick.editors.js |  512 ---
 .../js/jquery/slickgrid/slick.formatters.js     |   59 -
 .../webapp/js/jquery/slickgrid/slick.grid.js    | 3422 -----------------
 .../webapp/js/jquery/tabbs/jquery.tabbs.css     |   48 -
 .../main/webapp/js/jquery/tabbs/jquery.tabbs.js |   96 -
 .../ui-smoothness/images/animated-overlay.gif   |  Bin 1738 -> 0 bytes
 .../images/ui-bg_flat_0_aaaaaa_40x100.png       |  Bin 212 -> 0 bytes
 .../images/ui-bg_flat_75_ffffff_40x100.png      |  Bin 208 -> 0 bytes
 .../images/ui-bg_glass_55_fbf9ee_1x400.png      |  Bin 335 -> 0 bytes
 .../images/ui-bg_glass_65_ffffff_1x400.png      |  Bin 207 -> 0 bytes
 .../images/ui-bg_glass_75_dadada_1x400.png      |  Bin 262 -> 0 bytes
 .../images/ui-bg_glass_75_e6e6e6_1x400.png      |  Bin 262 -> 0 bytes
 .../images/ui-bg_glass_95_fef1ec_1x400.png      |  Bin 332 -> 0 bytes
 .../ui-bg_highlight-soft_75_cccccc_1x100.png    |  Bin 280 -> 0 bytes
 .../images/ui-icons_222222_256x240.png          |  Bin 6922 -> 0 bytes
 .../images/ui-icons_2e83ff_256x240.png          |  Bin 4549 -> 0 bytes
 .../images/ui-icons_454545_256x240.png          |  Bin 6992 -> 0 bytes
 .../images/ui-icons_888888_256x240.png          |  Bin 6999 -> 0 bytes
 .../images/ui-icons_cd0a0a_256x240.png          |  Bin 4549 -> 0 bytes
 .../ui-smoothness/jquery-ui-1.10.4.min.css      |    7 -
 .../ui-smoothness/jquery-ui-1.10.4.min.js       |    7 -
 .../web/nifi-web-ui/src/main/webapp/js/json2.js |  475 ---
 .../js/nf/bulletin-board/nf-bulletin-board.js   |  422 --
 .../src/main/webapp/js/nf/canvas/nf-actions.js  | 1150 ------
 .../src/main/webapp/js/nf/canvas/nf-birdseye.js |  340 --
 .../webapp/js/nf/canvas/nf-canvas-header.js     |  293 --
 .../webapp/js/nf/canvas/nf-canvas-toolbar.js    |  174 -
 .../webapp/js/nf/canvas/nf-canvas-toolbox.js    | 1211 ------
 .../main/webapp/js/nf/canvas/nf-canvas-utils.js | 1307 -------
 .../src/main/webapp/js/nf/canvas/nf-canvas.js   | 1530 --------
 .../main/webapp/js/nf/canvas/nf-clipboard.js    |   96 -
 .../main/webapp/js/nf/canvas/nf-connectable.js  |  210 -
 .../js/nf/canvas/nf-connection-configuration.js | 1381 -------
 .../main/webapp/js/nf/canvas/nf-connection.js   | 1533 --------
 .../main/webapp/js/nf/canvas/nf-context-menu.js |  452 ---
 .../js/nf/canvas/nf-custom-processor-ui.js      |   43 -
 .../main/webapp/js/nf/canvas/nf-draggable.js    |  319 --
 .../src/main/webapp/js/nf/canvas/nf-funnel.js   |  279 --
 .../src/main/webapp/js/nf/canvas/nf-go-to.js    |  730 ----
 .../webapp/js/nf/canvas/nf-graph-control.js     |  137 -
 .../src/main/webapp/js/nf/canvas/nf-graph.js    |  194 -
 .../js/nf/canvas/nf-label-configuration.js      |  132 -
 .../src/main/webapp/js/nf/canvas/nf-label.js    |  559 ---
 .../js/nf/canvas/nf-port-configuration.js       |  170 -
 .../main/webapp/js/nf/canvas/nf-port-details.js |   63 -
 .../src/main/webapp/js/nf/canvas/nf-port.js     |  632 ---
 .../nf/canvas/nf-process-group-configuration.js |  107 -
 .../js/nf/canvas/nf-process-group-details.js    |   61 -
 .../webapp/js/nf/canvas/nf-process-group.js     | 1064 -----
 .../js/nf/canvas/nf-processor-configuration.js  |  734 ----
 .../nf-processor-property-combo-editor.js       |  175 -
 .../canvas/nf-processor-property-nfel-editor.js |  207 -
 .../js/nf/canvas/nf-processor-property-table.js |  567 ---
 .../canvas/nf-processor-property-text-editor.js |  212 -
 .../main/webapp/js/nf/canvas/nf-processor.js    |  831 ----
 .../main/webapp/js/nf/canvas/nf-registration.js |   68 -
 .../nf-remote-process-group-configuration.js    |  124 -
 .../canvas/nf-remote-process-group-details.js   |   67 -
 .../nf/canvas/nf-remote-process-group-ports.js  |  525 ---
 .../js/nf/canvas/nf-remote-process-group.js     | 1060 -----
 .../src/main/webapp/js/nf/canvas/nf-search.js   |  179 -
 .../nf/canvas/nf-secure-port-configuration.js   |  384 --
 .../js/nf/canvas/nf-secure-port-details.js      |  118 -
 .../main/webapp/js/nf/canvas/nf-selectable.js   |   58 -
 .../src/main/webapp/js/nf/canvas/nf-settings.js |  146 -
 .../src/main/webapp/js/nf/canvas/nf-snippet.js  |  212 -
 .../src/main/webapp/js/nf/canvas/nf-storage.js  |  136 -
 .../webapp/js/nf/canvas/nf-toolbar-action.js    |   86 -
 .../webapp/js/nf/cluster/nf-cluster-table.js    |  620 ---
 .../src/main/webapp/js/nf/cluster/nf-cluster.js |  154 -
 .../webapp/js/nf/counters/nf-counters-table.js  |  294 --
 .../main/webapp/js/nf/counters/nf-counters.js   |  153 -
 .../webapp/js/nf/history/nf-history-model.js    |  218 --
 .../webapp/js/nf/history/nf-history-table.js    |  429 ---
 .../src/main/webapp/js/nf/history/nf-history.js |  154 -
 .../src/main/webapp/js/nf/nf-client.js          |   56 -
 .../src/main/webapp/js/nf/nf-common.js          |  899 -----
 .../main/webapp/js/nf/nf-connection-details.js  |  467 ---
 .../src/main/webapp/js/nf/nf-dialog.js          |  128 -
 .../src/main/webapp/js/nf/nf-namespace.js       |   20 -
 .../main/webapp/js/nf/nf-processor-details.js   |  586 ---
 .../src/main/webapp/js/nf/nf-shell.js           |  167 -
 .../src/main/webapp/js/nf/nf-status-history.js  | 1320 -------
 .../js/nf/provenance/nf-provenance-lineage.js   | 1413 -------
 .../js/nf/provenance/nf-provenance-table.js     | 1371 -------
 .../webapp/js/nf/provenance/nf-provenance.js    |  204 -
 .../webapp/js/nf/summary/nf-cluster-search.js   |  183 -
 .../webapp/js/nf/summary/nf-summary-table.js    | 2436 ------------
 .../src/main/webapp/js/nf/summary/nf-summary.js |  174 -
 .../js/nf/templates/nf-templates-table.js       |  332 --
 .../main/webapp/js/nf/templates/nf-templates.js |  235 --
 .../main/webapp/js/nf/users/nf-users-table.js   | 1090 ------
 .../src/main/webapp/js/nf/users/nf-users.js     |  149 -
 .../framework-bundle/framework/web/pom.xml      |   74 -
 .../web/web-optimistic-locking/.gitignore       |    1 -
 .../web/web-optimistic-locking/pom.xml          |   32 -
 .../apache/nifi/web/ConfigurationSnapshot.java  |   66 -
 .../nifi/web/OptimisticLockingManager.java      |   95 -
 .../web/StandardOptimisticLockingManager.java   |   77 -
 .../framework/web/web-security/.gitignore       |    1 -
 .../framework/web/web-security/pom.xml          |   81 -
 .../org/apache/nifi/web/security/DnUtils.java   |   87 -
 .../web/security/UntrustedProxyException.java   |   34 -
 .../anonymous/NiFiAnonymousUserFilter.java      |  101 -
 .../NiFiAuthenticationEntryPoint.java           |   68 -
 .../authorization/NiFiAuthorizationService.java |  173 -
 .../authorization/NodeAuthorizedUserFilter.java |  128 -
 .../nifi/web/security/user/NiFiUserDetails.java |  110 -
 .../nifi/web/security/user/NiFiUserUtils.java   |   82 -
 .../x509/SubjectDnX509PrincipalExtractor.java   |   38 -
 .../security/x509/X509AuthenticationFilter.java |  349 --
 .../security/x509/X509CertificateExtractor.java |   52 -
 .../x509/ocsp/CertificateStatusException.java   |   42 -
 .../x509/ocsp/OcspCertificateValidator.java     |  444 ---
 .../web/security/x509/ocsp/OcspRequest.java     |   69 -
 .../nifi/web/security/x509/ocsp/OcspStatus.java |   82 -
 .../resources/nifi-web-security-context.xml     |   83 -
 .../NiFiAuthorizationServiceTest.java           |  250 --
 nar-bundles/framework-bundle/nar/.gitignore     |    1 -
 nar-bundles/framework-bundle/nar/pom.xml        |   66 -
 nar-bundles/framework-bundle/pom.xml            |  114 -
 .../hadoop-bundle/hdfs-processors/pom.xml       |   65 -
 .../hadoop/AbstractHadoopProcessor.java         |  221 --
 .../hadoop/CreateHadoopSequenceFile.java        |  177 -
 ...lowFileStreamUnpackerSequenceFileWriter.java |  170 -
 .../apache/nifi/processors/hadoop/GetHDFS.java  |  563 ---
 .../processors/hadoop/GetHDFSSequenceFile.java  |  146 -
 .../nifi/processors/hadoop/KeyValueReader.java  |  132 -
 .../apache/nifi/processors/hadoop/PutHDFS.java  |  403 --
 .../hadoop/SequenceFileWriterImpl.java          |  119 -
 .../hadoop/TarUnpackerSequenceFileWriter.java   |   54 -
 .../nifi/processors/hadoop/ValueReader.java     |  116 -
 .../hadoop/ZipUnpackerSequenceFileWriter.java   |   57 -
 .../hadoop/util/ByteFilteringOutputStream.java  |  163 -
 .../hadoop/util/InputStreamWritable.java        |   61 -
 .../hadoop/util/OutputStreamWritable.java       |   85 -
 .../hadoop/util/SequenceFileReader.java         |   28 -
 .../hadoop/util/SequenceFileWriter.java         |   40 -
 .../org.apache.nifi.processor.Processor         |   18 -
 .../index.html                                  |   88 -
 .../index.html                                  |  162 -
 .../index.html                                  |  150 -
 .../index.html                                  |  159 -
 .../processors/hadoop/AbstractHadoopTest.java   |   90 -
 .../nifi/processors/hadoop/GetHDFSTest.java     |  123 -
 .../nifi/processors/hadoop/PutHDFSTest.java     |  158 -
 .../hadoop/SimpleHadoopProcessor.java           |   30 -
 .../hadoop/TestCreateHadoopSequenceFile.java    |  176 -
 .../src/test/resources/core-site-broken.xml     |   25 -
 .../src/test/resources/core-site.xml            |   25 -
 .../resources/testdata/13545312236534130.tar    |  Bin 1505280 -> 0 bytes
 .../resources/testdata/13545423550275052.zip    |  Bin 1500841 -> 0 bytes
 .../resources/testdata/13545479542069498.pkg    |  Bin 1500429 -> 0 bytes
 .../src/test/resources/testdata/randombytes-1   |  Bin 500000 -> 0 bytes
 .../src/test/resources/testdata/randombytes-2   |  Bin 500000 -> 0 bytes
 .../src/test/resources/testdata/randombytes-3   |  Bin 500000 -> 0 bytes
 nar-bundles/hadoop-bundle/nar/pom.xml           |   41 -
 nar-bundles/hadoop-bundle/pom.xml               |   46 -
 nar-bundles/hadoop-libraries-bundle/nar/pom.xml |   45 -
 nar-bundles/hadoop-libraries-bundle/pom.xml     |   36 -
 nar-bundles/jetty-bundle/pom.xml                |   82 -
 nar-bundles/kafka-bundle/kafka-nar/pom.xml      |   37 -
 .../kafka-bundle/kafka-processors/pom.xml       |   76 -
 .../apache/nifi/processors/kafka/GetKafka.java  |  330 --
 .../apache/nifi/processors/kafka/PutKafka.java  |  419 --
 .../org.apache.nifi.processor.Processor         |   16 -
 .../index.html                                  |  173 -
 .../index.html                                  |  189 -
 .../nifi/processors/kafka/TestGetKafka.java     |  162 -
 .../nifi/processors/kafka/TestPutKafka.java     |  236 --
 nar-bundles/kafka-bundle/pom.xml                |   35 -
 .../monitor-threshold-bundle/nar/pom.xml        |   39 -
 nar-bundles/monitor-threshold-bundle/pom.xml    |   51 -
 .../monitor-threshold-bundle/processor/pom.xml  |   45 -
 .../processors/monitor/MonitorThreshold.java    |  904 -----
 .../processors/monitor/ThresholdsParser.java    |  134 -
 .../org.apache.nifi.processor.Processor         |   15 -
 .../index.html                                  |  485 ---
 .../monitor/TestMonitorThreshold.java           | 1560 --------
 .../processor/src/test/resources/testFile       |   23 -
 .../resources/threshold_settings_allow_0.xml    |   21 -
 .../resources/threshold_settings_allow_1.xml    |   21 -
 .../threshold_settings_allow_10_KB.xml          |   21 -
 .../threshold_settings_allow_10_KB_and_2_KB.xml |   26 -
 .../threshold_settings_allow_1669_bytes.xml     |   21 -
 .../resources/threshold_settings_allow_2.xml    |   21 -
 .../threshold_settings_allow_2_default.xml      |   21 -
 ...eshold_settings_allow_3336_bytes_default.xml |   21 -
 ...hold_settings_with_empty_attribute_value.xml |   21 -
 ...ettings_with_spaces_for_attribute_values.xml |   22 -
 .../ui/nb-configuration.xml                     |   18 -
 nar-bundles/monitor-threshold-bundle/ui/pom.xml |  134 -
 .../thresholds/ui/AttributeComparator.java      |   36 -
 .../thresholds/ui/AttributeResource.java        |  242 --
 .../monitor/thresholds/ui/RuleComparator.java   |   46 -
 .../thresholds/ui/ThresholdResource.java        |  227 --
 .../ui/ThresholdSettingsResource.java           |  191 -
 .../thresholds/ui/ThresholdsConfigFile.java     |  296 --
 .../ui/src/main/webapp/META-INF/nifi-processor  |   15 -
 .../webapp/WEB-INF/jsp/attribute-add-dialog.jsp |   43 -
 .../WEB-INF/jsp/attribute-confirm-dialog.jsp    |   21 -
 .../WEB-INF/jsp/attribute-edit-dialog.jsp       |   27 -
 .../WEB-INF/jsp/attribute-filter-dialog.jsp     |   26 -
 .../main/webapp/WEB-INF/jsp/error-dialog.jsp    |   21 -
 .../ui/src/main/webapp/WEB-INF/jsp/index.jsp    |  101 -
 .../webapp/WEB-INF/jsp/threshold-add-dialog.jsp |   45 -
 .../WEB-INF/jsp/threshold-confirm-dialog.jsp    |   21 -
 .../webapp/WEB-INF/jsp/threshold-dialog.jsp     |   48 -
 .../WEB-INF/jsp/threshold-edit-dialog.jsp       |   45 -
 .../WEB-INF/jsp/threshold-filter-dialog.jsp     |   40 -
 .../ui/src/main/webapp/WEB-INF/web.xml          |   48 -
 .../ui/src/main/webapp/css/threshold_styles.css |  422 --
 .../src/main/webapp/images/addWorksheetRow.png  |  Bin 323 -> 0 bytes
 .../ui/src/main/webapp/images/clear.png         |  Bin 912 -> 0 bytes
 .../ui/src/main/webapp/images/filter.gif        |  Bin 623 -> 0 bytes
 .../ui/src/main/webapp/images/firstPage.gif     |  Bin 577 -> 0 bytes
 .../main/webapp/images/firstPageDisabled.gif    |  Bin 365 -> 0 bytes
 .../ui/src/main/webapp/images/lastPage.gif      |  Bin 579 -> 0 bytes
 .../src/main/webapp/images/lastPageDisabled.gif |  Bin 368 -> 0 bytes
 .../ui/src/main/webapp/images/nextPage.gif      |  Bin 570 -> 0 bytes
 .../src/main/webapp/images/nextPageDisabled.gif |  Bin 362 -> 0 bytes
 .../ui/src/main/webapp/images/prevPage.gif      |  Bin 567 -> 0 bytes
 .../src/main/webapp/images/prevPageDisabled.gif |  Bin 361 -> 0 bytes
 .../main/webapp/images/removeWorksheetRow.png   |  Bin 655 -> 0 bytes
 .../ui/src/main/webapp/images/separator.gif     |  Bin 63 -> 0 bytes
 .../webapp/js/jquery/jqgrid/css/ui.jqgrid.css   |  171 -
 .../js/jquery/jqgrid/js/i18n/grid.locale-en.js  |  170 -
 .../js/jquery/jqgrid/js/jquery.jqGrid.min.js    |  545 ---
 .../ui/src/main/webapp/js/nf-common.js          |  888 -----
 .../ui/src/main/xsd/threshold_settings.xsd      |   54 -
 .../nar/pom.xml                                 |   36 -
 .../persistent-provenance-repository/pom.xml    |   58 -
 .../nifi/provenance/IndexConfiguration.java     |  398 --
 .../PersistentProvenanceRepository.java         | 1892 ---------
 .../provenance/RepositoryConfiguration.java     |  299 --
 .../nifi/provenance/StandardRecordReader.java   |  305 --
 .../nifi/provenance/StandardRecordWriter.java   |  237 --
 .../provenance/expiration/ExpirationAction.java |   35 -
 .../expiration/FileRemovalAction.java           |   49 -
 .../provenance/lucene/DeleteIndexAction.java    |  139 -
 .../nifi/provenance/lucene/DocsReader.java      |  133 -
 .../nifi/provenance/lucene/FieldNames.java      |   23 -
 .../nifi/provenance/lucene/IndexSearch.java     |   84 -
 .../nifi/provenance/lucene/IndexingAction.java  |  219 --
 .../nifi/provenance/lucene/LineageQuery.java    |  108 -
 .../nifi/provenance/lucene/LuceneUtil.java      |  141 -
 .../provenance/rollover/CompressionAction.java  |   59 -
 .../provenance/rollover/RolloverAction.java     |   35 -
 .../provenance/serialization/RecordReader.java  |   31 -
 .../provenance/serialization/RecordReaders.java |   70 -
 .../provenance/serialization/RecordWriter.java  |   85 -
 .../provenance/serialization/RecordWriters.java |   30 -
 ...he.nifi.provenance.ProvenanceEventRepository |   15 -
 .../TestPersistentProvenanceRepository.java     | 1118 ------
 .../pom.xml                                     |   42 -
 nar-bundles/pom.xml                             |  134 -
 .../standard-bundle/jms-processors/pom.xml      |   55 -
 .../apache/nifi/processors/jms/GetJMSQueue.java |   75 -
 .../apache/nifi/processors/jms/GetJMSTopic.java |  359 --
 .../apache/nifi/processors/jms/JmsConsumer.java |  208 -
 .../org/apache/nifi/processors/jms/PutJMS.java  |  374 --
 .../nifi/processors/jms/util/JmsFactory.java    |  435 ---
 .../nifi/processors/jms/util/JmsProperties.java |  176 -
 .../jms/util/WrappedMessageConsumer.java        |   77 -
 .../jms/util/WrappedMessageProducer.java        |   77 -
 .../org.apache.nifi.processor.Processor         |   17 -
 .../index.html                                  |  118 -
 .../index.html                                  |  122 -
 .../index.html                                  |  152 -
 .../nifi/processors/jms/GetJMSQueueTest.java    |  141 -
 nar-bundles/standard-bundle/nar/pom.xml         |   57 -
 nar-bundles/standard-bundle/pom.xml             |   71 -
 .../standard-ganglia-reporter/pom.xml           |   52 -
 .../ganglia/StandardGangliaReporter.java        |  262 --
 .../org.apache.nifi.reporting.ReportingTask     |   15 -
 .../index.html                                  |   65 -
 .../standard-prioritizers/pom.xml               |   40 -
 .../prioritizer/FirstInFirstOutPrioritizer.java |   37 -
 .../NewestFlowFileFirstPrioritizer.java         |   37 -
 .../OldestFlowFileFirstPrioritizer.java         |   37 -
 .../PriorityAttributePrioritizer.java           |   87 -
 ...org.apache.nifi.flowfile.FlowFilePrioritizer |   18 -
 .../prioritizer/NewestFirstPrioritizerTest.java |   66 -
 .../prioritizer/OldestFirstPrioritizerTest.java |   66 -
 .../PriorityAttributePrioritizerTest.java       |  117 -
 .../standard-bundle/standard-processors/pom.xml |  154 -
 .../standard/Base64EncodeContent.java           |  142 -
 .../processors/standard/CompressContent.java    |  307 --
 .../nifi/processors/standard/ControlRate.java   |  381 --
 .../standard/ConvertCharacterSet.java           |  175 -
 .../processors/standard/DetectDuplicate.java    |  249 --
 .../processors/standard/DistributeLoad.java     |  498 ---
 .../processors/standard/EncryptContent.java     |  263 --
 .../standard/EvaluateRegularExpression.java     |  294 --
 .../nifi/processors/standard/EvaluateXPath.java |  429 ---
 .../processors/standard/EvaluateXQuery.java     |  463 ---
 .../standard/ExecuteStreamCommand.java          |  358 --
 .../processors/standard/GenerateFlowFile.java   |  164 -
 .../apache/nifi/processors/standard/GetFTP.java |   72 -
 .../nifi/processors/standard/GetFile.java       |  456 ---
 .../processors/standard/GetFileTransfer.java    |  301 --
 .../nifi/processors/standard/GetHTTP.java       |  480 ---
 .../nifi/processors/standard/GetSFTP.java       |   92 -
 .../nifi/processors/standard/HashAttribute.java |  255 --
 .../nifi/processors/standard/HashContent.java   |  151 -
 .../processors/standard/IdentifyMimeType.java   |  474 ---
 .../nifi/processors/standard/InvokeHTTP.java    |  758 ----
 .../nifi/processors/standard/ListenHTTP.java    |  321 --
 .../nifi/processors/standard/ListenUDP.java     |  627 ---
 .../nifi/processors/standard/LogAttribute.java  |  262 --
 .../nifi/processors/standard/MergeContent.java  | 1015 -----
 .../nifi/processors/standard/ModifyBytes.java   |  134 -
 .../processors/standard/MonitorActivity.java    |  206 -
 .../nifi/processors/standard/PostHTTP.java      |  904 -----
 .../nifi/processors/standard/PutEmail.java      |  297 --
 .../apache/nifi/processors/standard/PutFTP.java |  141 -
 .../nifi/processors/standard/PutFile.java       |  367 --
 .../processors/standard/PutFileTransfer.java    |  293 --
 .../nifi/processors/standard/PutSFTP.java       |   85 -
 .../nifi/processors/standard/ReplaceText.java   |  289 --
 .../standard/ReplaceTextWithMapping.java        |  383 --
 .../processors/standard/RouteOnAttribute.java   |  261 --
 .../processors/standard/RouteOnContent.java     |  232 --
 .../nifi/processors/standard/ScanAttribute.java |  229 --
 .../nifi/processors/standard/ScanContent.java   |  292 --
 .../processors/standard/SegmentContent.java     |  163 -
 .../nifi/processors/standard/SplitContent.java  |  260 --
 .../nifi/processors/standard/SplitText.java     |  370 --
 .../nifi/processors/standard/SplitXml.java      |  300 --
 .../nifi/processors/standard/TransformXml.java  |  194 -
 .../nifi/processors/standard/UnpackContent.java |  427 ---
 .../nifi/processors/standard/ValidateXml.java   |  147 -
 .../servlets/ContentAcknowledgmentServlet.java  |  136 -
 .../standard/servlets/ListenHTTPServlet.java    |  320 --
 .../nifi/processors/standard/util/Bin.java      |  168 -
 .../processors/standard/util/BinManager.java    |  241 --
 .../standard/util/DocumentReaderCallback.java   |   74 -
 .../processors/standard/util/FTPTransfer.java   |  541 ---
 .../nifi/processors/standard/util/FTPUtils.java |  322 --
 .../nifi/processors/standard/util/FileInfo.java |  167 -
 .../processors/standard/util/FileTransfer.java  |  244 --
 .../standard/util/FlowFileSessionWrapper.java   |   44 -
 .../standard/util/NLKBufferedReader.java        |  187 -
 .../standard/util/SFTPConnection.java           |   82 -
 .../processors/standard/util/SFTPTransfer.java  |  637 ---
 .../processors/standard/util/SFTPUtils.java     |  324 --
 .../standard/util/UDPStreamConsumer.java        |  214 --
 .../util/ValidatingBase64InputStream.java       |   76 -
 .../standard/util/XmlElementNotifier.java       |   22 -
 .../standard/util/XmlSplitterSaxParser.java     |  112 -
 .../org.apache.nifi.processor.Processor         |   58 -
 .../index.html                                  |   63 -
 .../index.html                                  |  166 -
 .../index.html                                  |  116 -
 .../index.html                                  |   65 -
 .../index.html                                  |  147 -
 .../index.html                                  |  106 -
 .../index.html                                  |   97 -
 .../index.html                                  |  160 -
 .../index.html                                  |  135 -
 .../index.html                                  |  311 --
 .../index.html                                  |  111 -
 .../index.html                                  |   64 -
 .../index.html                                  |  227 --
 .../index.html                                  |  186 -
 .../index.html                                  |  143 -
 .../index.html                                  |  250 --
 .../index.html                                  |   88 -
 .../index.html                                  |   89 -
 .../index.html                                  |  136 -
 .../index.html                                  |  181 -
 .../index.html                                  |   86 -
 .../index.html                                  |  144 -
 .../index.html                                  |   80 -
 .../index.html                                  |  347 --
 .../index.html                                  |   64 -
 .../index.html                                  |  143 -
 .../index.html                                  |  187 -
 .../index.html                                  |  114 -
 .../index.html                                  |  283 --
 .../index.html                                  |  109 -
 .../index.html                                  |  281 --
 .../index.html                                  |   91 -
 .../index.html                                  |  114 -
 .../index.html                                  |  110 -
 .../index.html                                  |   82 -
 .../index.html                                  |   85 -
 .../index.html                                  |  100 -
 .../index.html                                  |  123 -
 .../index.html                                  |  107 -
 .../index.html                                  |  121 -
 .../index.html                                  |   64 -
 .../index.html                                  |   63 -
 .../index.html                                  |  163 -
 .../index.html                                  |   56 -
 .../src/test/java/TestIngestAndUpdate.java      |   34 -
 .../src/test/java/TestSuccess.java              |   24 -
 .../processors/standard/HelloWorldServlet.java  |   41 -
 .../standard/RESTServiceContentModified.java    |   78 -
 .../standard/TestBase64EncodeContent.java       |   83 -
 .../standard/TestCompressContent.java           |  111 -
 .../processors/standard/TestControlRate.java    |   68 -
 .../standard/TestConvertCharacterSet.java       |   47 -
 .../standard/TestDetectDuplicate.java           |  206 -
 .../processors/standard/TestDistributeLoad.java |  138 -
 .../processors/standard/TestEncryptContent.java |   65 -
 .../standard/TestEvaluateRegularExpression.java |  319 --
 .../processors/standard/TestEvaluateXPath.java  |  159 -
 .../processors/standard/TestEvaluateXQuery.java |  651 ----
 .../standard/TestExecuteStreamCommand.java      |  188 -
 .../nifi/processors/standard/TestGetFile.java   |  186 -
 .../nifi/processors/standard/TestGetHTTP.java   |  354 --
 .../processors/standard/TestHashAttribute.java  |   99 -
 .../processors/standard/TestHashContent.java    |   67 -
 .../standard/TestIdentifyMimeType.java          |  131 -
 .../processors/standard/TestInvokeHTTP.java     |  593 ---
 .../nifi/processors/standard/TestListenUDP.java |  214 --
 .../processors/standard/TestMergeContent.java   |  593 ---
 .../processors/standard/TestModifyBytes.java    |  190 -
 .../standard/TestMonitorActivity.java           |  193 -
 .../processors/standard/TestReplaceText.java    |  371 --
 .../standard/TestReplaceTextLineByLine.java     |  337 --
 .../standard/TestReplaceTextWithMapping.java    |  332 --
 .../standard/TestRouteOnAttribute.java          |  161 -
 .../processors/standard/TestRouteOnContent.java |   73 -
 .../processors/standard/TestScanAttribute.java  |  149 -
 .../processors/standard/TestScanContent.java    |   85 -
 .../processors/standard/TestSegmentContent.java |   65 -
 .../nifi/processors/standard/TestServer.java    |  177 -
 .../processors/standard/TestSplitContent.java   |  235 --
 .../nifi/processors/standard/TestSplitText.java |  173 -
 .../nifi/processors/standard/TestSplitXml.java  |  103 -
 .../processors/standard/TestTransformXml.java   |  120 -
 .../processors/standard/TestUnpackContent.java  |  226 --
 .../processors/standard/TestValidateXml.java    |   42 -
 .../standard/UserAgentTestingServlet.java       |   42 -
 .../CharacterSetConversionSamples/Converted.txt |    1 -
 .../Converted2.txt                              |  Bin 134580 -> 0 bytes
 .../CharacterSetConversionSamples/Original.txt  |  418 --
 .../resources/CompressedData/SampleFile.txt     |  418 --
 .../resources/CompressedData/SampleFile.txt.bz2 |  Bin 388 -> 0 bytes
 .../resources/CompressedData/SampleFile.txt.gz  |  Bin 321 -> 0 bytes
 .../CompressedData/SampleFile1.txt.bz2          |  Bin 154 -> 0 bytes
 .../resources/CompressedData/SampleFile1.txt.gz |  Bin 321 -> 0 bytes
 .../CompressedData/SampleFileConcat.txt         |  835 ----
 .../CompressedData/SampleFileConcat.txt.bz2     |  Bin 542 -> 0 bytes
 .../test/resources/ExecuteCommand/1000bytes.txt |    1 -
 .../ExecuteCommand/TestIngestAndUpdate.jar      |  Bin 1170 -> 0 bytes
 .../resources/ExecuteCommand/TestSuccess.jar    |  Bin 827 -> 0 bytes
 .../src/test/resources/ExecuteCommand/test.txt  |    2 -
 .../dictionary-with-empty-new-lines             |    5 -
 .../ScanAttribute/dictionary-with-extra-info    |    9 -
 .../test/resources/ScanAttribute/dictionary1    |    5 -
 .../test/resources/TestIdentifyMimeType/1.7z    |  Bin 133 -> 0 bytes
 .../test/resources/TestIdentifyMimeType/1.jar   |  Bin 466 -> 0 bytes
 .../test/resources/TestIdentifyMimeType/1.mdb   |  Bin 237568 -> 0 bytes
 .../test/resources/TestIdentifyMimeType/1.pdf   |  Bin 277032 -> 0 bytes
 .../test/resources/TestIdentifyMimeType/1.tar   |  Bin 2048 -> 0 bytes
 .../resources/TestIdentifyMimeType/1.txt.bz2    |  Bin 59 -> 0 bytes
 .../resources/TestIdentifyMimeType/1.txt.gz     |  Bin 50 -> 0 bytes
 .../test/resources/TestIdentifyMimeType/1.xml   |   20 -
 .../test/resources/TestIdentifyMimeType/1.zip   |  Bin 165 -> 0 bytes
 .../TestIdentifyMimeType/bgBannerFoot.png       |  Bin 189 -> 0 bytes
 .../TestIdentifyMimeType/blueBtnBg.jpg          |  Bin 356 -> 0 bytes
 .../TestIdentifyMimeType/flowfilev1.tar         |  Bin 10240 -> 0 bytes
 .../resources/TestIdentifyMimeType/flowfilev3   |  Bin 40 -> 0 bytes
 .../resources/TestIdentifyMimeType/grid.gif     |  Bin 135 -> 0 bytes
 .../test/resources/TestMergeContent/demarcate   |    1 -
 .../src/test/resources/TestMergeContent/foot    |    1 -
 .../src/test/resources/TestMergeContent/head    |    1 -
 .../test/resources/TestModifyBytes/noFooter.txt |   10 -
 .../TestModifyBytes/noFooter_noHeader.txt       |   10 -
 .../test/resources/TestModifyBytes/noHeader.txt |   11 -
 .../test/resources/TestModifyBytes/testFile.txt |   11 -
 .../TestReplaceTextLineByLine/$1$1.txt          |   11 -
 .../BRue_cRue_RiRey.txt                         |   11 -
 .../TestReplaceTextLineByLine/Blu$2e_clu$2e.txt |   11 -
 .../TestReplaceTextLineByLine/D$d_h$d.txt       |   11 -
 .../TestReplaceTextLineByLine/Good.txt          |    1 -
 .../TestReplaceTextLineByLine/Spider.txt        |   11 -
 .../TestReplaceTextLineByLine/[DODO].txt        |   11 -
 .../TestReplaceTextLineByLine/cu[$1]_Po[$1].txt |   11 -
 .../TestReplaceTextLineByLine/cu_Po.txt         |   11 -
 .../TestReplaceTextLineByLine/food.txt          |   11 -
 .../TestReplaceTextLineByLine/testFile.txt      |   11 -
 .../color-fruit-backreference-mapping.txt       |    7 -
 .../color-fruit-blank-mapping.txt               |    7 -
 .../color-fruit-escaped-dollar-mapping.txt      |    7 -
 ...t-excessive-backreference-mapping-simple.txt |    6 -
 ...or-fruit-excessive-backreference-mapping.txt |    6 -
 ...olor-fruit-invalid-backreference-mapping.txt |    7 -
 .../color-fruit-mapping.txt                     |    7 -
 .../color-fruit-no-match-mapping.txt            |    7 -
 .../color-fruit-space-mapping.txt               |    6 -
 .../colors-without-dashes.txt                   |    4 -
 .../TestReplaceTextWithMapping/colors.txt       |    4 -
 .../test/resources/TestScanContent/helloWorld   |    1 -
 .../resources/TestScanContent/wellthengood-bye  |    1 -
 .../src/test/resources/TestSplitText/1.txt      |    5 -
 .../src/test/resources/TestSplitText/2.txt      |    5 -
 .../src/test/resources/TestSplitText/3.txt      |    5 -
 .../src/test/resources/TestSplitText/4.txt      |    3 -
 .../src/test/resources/TestSplitText/5.txt      |    7 -
 .../src/test/resources/TestSplitText/6.txt      |    7 -
 .../test/resources/TestSplitText/original.txt   |   12 -
 .../test/resources/TestTransformXml/math.html   |    8 -
 .../test/resources/TestTransformXml/math.xml    |   21 -
 .../test/resources/TestTransformXml/math.xsl    |   36 -
 .../test/resources/TestTransformXml/tokens.csv  |    2 -
 .../test/resources/TestTransformXml/tokens.xml  |   17 -
 .../test/resources/TestTransformXml/tokens.xsl  |  103 -
 .../resources/TestUnpackContent/data.flowfilev2 |  Bin 255 -> 0 bytes
 .../resources/TestUnpackContent/data.flowfilev3 |  Bin 357 -> 0 bytes
 .../test/resources/TestUnpackContent/data.tar   |  Bin 3584 -> 0 bytes
 .../test/resources/TestUnpackContent/data.zip   |  Bin 359 -> 0 bytes
 .../resources/TestUnpackContent/folder/cal.txt  |    8 -
 .../resources/TestUnpackContent/folder/date.txt |    1 -
 .../src/test/resources/TestXml/XmlBundle.xsd    |   34 -
 .../src/test/resources/TestXml/fruit.xml        |   47 -
 .../src/test/resources/TestXml/subNode.xml      |   21 -
 .../src/test/resources/TestXml/xml-bundle-1     |   51 -
 .../src/test/resources/TestXml/xml-snippet.xml  |   25 -
 .../src/test/resources/hello.txt                |    1 -
 .../src/test/resources/localhost-ks.jks         |  Bin 1298 -> 0 bytes
 .../src/test/resources/localhost-ts.jks         |  Bin 891 -> 0 bytes
 .../src/test/resources/localhost.cer            |  Bin 829 -> 0 bytes
 .../src/test/resources/logback-test.xml         |   49 -
 .../standard-reporting-tasks/pom.xml            |   60 -
 .../ControllerStatusReportingTask.java          |  347 --
 .../nifi/controller/MonitorDiskUsage.java       |  113 -
 .../apache/nifi/controller/MonitorMemory.java   |  260 --
 .../org.apache.nifi.reporting.ReportingTask     |   17 -
 .../index.html                                  |   85 -
 .../index.html                                  |   58 -
 .../index.html                                  |   77 -
 .../nifi/controller/TestMonitorDiskUsage.java   |   61 -
 .../pom.xml                                     |   36 -
 .../distributed/cache/client/Deserializer.java  |   41 -
 .../cache/client/DistributedMapCacheClient.java |  114 -
 .../cache/client/DistributedSetCacheClient.java |   74 -
 .../distributed/cache/client/Serializer.java    |   40 -
 .../exception/DeserializationException.java     |   33 -
 .../exception/SerializationException.java       |   33 -
 .../distributed-cache-client-service/pom.xml    |   56 -
 .../distributed/cache/client/CommsSession.java  |   46 -
 .../DistributedMapCacheClientService.java       |  305 --
 .../DistributedSetCacheClientService.java       |  215 --
 .../cache/client/SSLCommsSession.java           |  106 -
 .../cache/client/StandardCommsSession.java      |  124 -
 ...org.apache.nifi.controller.ControllerService |   16 -
 .../index.html                                  |   78 -
 .../index.html                                  |   51 -
 .../distributed-cache-protocol/pom.xml          |   39 -
 .../cache/protocol/ProtocolHandshake.java       |  119 -
 .../protocol/exception/HandshakeException.java  |   27 -
 .../distributed-cache-server/pom.xml            |   75 -
 .../cache/server/AbstractCacheServer.java       |  199 -
 .../distributed/cache/server/CacheRecord.java   |   57 -
 .../distributed/cache/server/CacheServer.java   |   26 -
 .../cache/server/DistributedCacheServer.java    |  107 -
 .../cache/server/DistributedSetCacheServer.java |   68 -
 .../cache/server/EvictionPolicy.java            |   73 -
 .../cache/server/SetCacheServer.java            |  104 -
 .../server/map/DistributedMapCacheServer.java   |   71 -
 .../distributed/cache/server/map/MapCache.java  |   29 -
 .../cache/server/map/MapCacheRecord.java        |   58 -
 .../cache/server/map/MapCacheServer.java        |  145 -
 .../cache/server/map/MapPutResult.java          |   59 -
 .../cache/server/map/PersistentMapCache.java    |  210 -
 .../cache/server/map/SimpleMapCache.java        |  165 -
 .../cache/server/set/PersistentSetCache.java    |  194 -
 .../distributed/cache/server/set/SetCache.java  |   29 -
 .../cache/server/set/SetCacheRecord.java        |   55 -
 .../cache/server/set/SetCacheResult.java        |   43 -
 .../cache/server/set/SimpleSetCache.java        |  117 -
 ...org.apache.nifi.controller.ControllerService |   16 -
 .../index.html                                  |   82 -
 .../cache/server/TestServerAndClient.java       |  535 ---
 .../src/test/resources/localhost-ks.jks         |  Bin 1298 -> 0 bytes
 .../src/test/resources/localhost-ts.jks         |  Bin 891 -> 0 bytes
 .../distributed-cache-services-nar/pom.xml      |   47 -
 .../distributed-cache-services-bundle/pom.xml   |   35 -
 .../load-distribution-service-api/pom.xml       |   37 -
 .../nifi/loading/LoadDistributionListener.java  |   24 -
 .../nifi/loading/LoadDistributionService.java   |   33 -
 nar-bundles/standard-services/pom.xml           |   39 -
 .../ssl-context-bundle/nar/pom.xml              |   39 -
 .../ssl-context-bundle/pom.xml                  |   34 -
 .../ssl-context-service/pom.xml                 |   52 -
 .../nifi/ssl/StandardSSLContextService.java     |  354 --
 ...org.apache.nifi.controller.ControllerService |   15 -
 .../index.html                                  |   63 -
 .../apache/nifi/ssl/SSLContextServiceTest.java  |  197 -
 .../java/org/apache/nifi/ssl/TestProcessor.java |   47 -
 .../src/test/resources/localhost-ks.jks         |  Bin 1298 -> 0 bytes
 .../src/test/resources/localhost-ts.jks         |  Bin 891 -> 0 bytes
 .../ssl-context-service-api/pom.xml             |   36 -
 .../org/apache/nifi/ssl/SSLContextService.java  |   55 -
 .../standard-services-api-nar/pom.xml           |   45 -
 .../update-attribute-bundle/model/pom.xml       |   27 -
 .../apache/nifi/update/attributes/Action.java   |   55 -
 .../nifi/update/attributes/Condition.java       |   49 -
 .../apache/nifi/update/attributes/Criteria.java |  141 -
 .../nifi/update/attributes/FlowFilePolicy.java  |   34 -
 .../org/apache/nifi/update/attributes/Rule.java |   65 -
 .../update/attributes/serde/CriteriaSerDe.java  |  128 -
 nar-bundles/update-attribute-bundle/nar/pom.xml |   48 -
 nar-bundles/update-attribute-bundle/pom.xml     |   56 -
 .../update-attribute-bundle/processor/pom.xml   |   54 -
 .../processors/attributes/UpdateAttribute.java  |  508 ---
 .../org.apache.nifi.processor.Processor         |   15 -
 .../index.html                                  |  253 --
 .../update/attributes/TestUpdateAttribute.java  |  425 --
 .../ui/nb-configuration.xml                     |   18 -
 nar-bundles/update-attribute-bundle/ui/pom.xml  |   91 -
 .../attributes/UpdateAttributeModelFactory.java |  127 -
 .../attributes/api/ObjectMapperResolver.java    |   51 -
 .../update/attributes/api/RuleResource.java     |  684 ----
 .../nifi/update/attributes/dto/ActionDTO.java   |   68 -
 .../update/attributes/dto/ConditionDTO.java     |   59 -
 .../nifi/update/attributes/dto/DtoFactory.java  |   68 -
 .../nifi/update/attributes/dto/RuleDTO.java     |   65 -
 .../update/attributes/entity/ActionEntity.java  |   65 -
 .../attributes/entity/ConditionEntity.java      |   65 -
 .../entity/EvaluationContextEntity.java         |   74 -
 .../update/attributes/entity/RuleEntity.java    |   65 -
 .../update/attributes/entity/RulesEntity.java   |   66 -
 .../ui/src/main/webapp/META-INF/nifi-processor  |   15 -
 .../src/main/webapp/WEB-INF/jsp/worksheet.jsp   |  188 -
 .../ui/src/main/webapp/WEB-INF/web.xml          |   46 -
 .../ui/src/main/webapp/css/main.css             |  426 ---
 .../ui/src/main/webapp/images/bgInputText.png   |  Bin 139 -> 0 bytes
 .../ui/src/main/webapp/images/buttonNew.png     |  Bin 590 -> 0 bytes
 .../ui/src/main/webapp/images/iconDelete.png    |  Bin 670 -> 0 bytes
 .../ui/src/main/webapp/images/iconInfo.png      |  Bin 550 -> 0 bytes
 .../ui/src/main/webapp/js/application.js        | 1915 ---------
 .../nar/pom.xml                                 |   36 -
 .../pom.xml                                     |   44 -
 .../volatile-provenance-repository/pom.xml      |   47 -
 .../VolatileProvenanceRepository.java           |  745 ----
 ...he.nifi.provenance.ProvenanceEventRepository |   15 -
 .../TestVolatileProvenanceRepository.java       |  178 -
 .../src/test/resources/nifi.properties          |   18 -
 nar-maven-plugin/pom.xml                        |  385 --
 .../src/main/java/nifi/NarMojo.java             |  613 ---
 .../resources/META-INF/plexus/components.xml    |   52 -
 nifi-api/.gitignore                             |    2 -
 nifi-api/pom.xml                                |   31 -
 .../apache/nifi/authorization/Authority.java    |   97 -
 .../nifi/authorization/AuthorityProvider.java   |  181 -
 .../AuthorityProviderConfigurationContext.java  |   52 -
 .../AuthorityProviderInitializationContext.java |   27 -
 .../authorization/AuthorityProviderLookup.java  |   25 -
 .../authorization/DownloadAuthorization.java    |   86 -
 .../annotation/AuthorityProviderContext.java    |   36 -
 .../exception/AuthorityAccessException.java     |   33 -
 .../IdentityAlreadyExistsException.java         |   32 -
 .../exception/ProviderCreationException.java    |   40 -
 .../exception/ProviderDestructionException.java |   40 -
 .../exception/UnknownIdentityException.java     |   32 -
 .../AbstractConfigurableComponent.java          |  221 --
 .../apache/nifi/components/AllowableValue.java  |  136 -
 .../nifi/components/ConfigurableComponent.java  |   77 -
 .../nifi/components/PropertyDescriptor.java     |  552 ---
 .../apache/nifi/components/PropertyValue.java   |  189 -
 .../nifi/components/ValidationContext.java      |   82 -
 .../nifi/components/ValidationResult.java       |  173 -
 .../org/apache/nifi/components/Validator.java   |   55 -
 .../controller/AbstractControllerService.java   |   91 -
 .../nifi/controller/ConfigurationContext.java   |   46 -
 .../nifi/controller/ControllerService.java      |  182 -
 .../ControllerServiceInitializationContext.java |   36 -
 .../controller/ControllerServiceLookup.java     |   66 -
 .../apache/nifi/controller/FlowFileQueue.java   |  200 -
 .../apache/nifi/controller/ScheduledState.java  |   37 -
 .../org/apache/nifi/controller/Snippet.java     |  113 -
 .../org/apache/nifi/controller/Triggerable.java |  116 -
 .../controller/annotation/OnConfigured.java     |   41 -
 .../repository/ContentRepository.java           |  314 --
 .../controller/repository/FlowFileRecord.java   |   54 -
 .../repository/FlowFileRepository.java          |  128 -
 .../repository/FlowFileSwapManager.java         |   63 -
 .../controller/repository/QueueProvider.java    |   35 -
 .../controller/repository/RepositoryRecord.java |  100 -
 .../repository/RepositoryRecordType.java        |   26 -
 .../repository/claim/ContentClaim.java          |   55 -
 .../repository/claim/ContentClaimManager.java   |  143 -
 .../controller/status/ConnectionStatus.java     |  193 -
 .../nifi/controller/status/PortStatus.java      |  201 -
 .../controller/status/ProcessGroupStatus.java   |  547 ---
 .../nifi/controller/status/ProcessorStatus.java |  275 --
 .../status/RemoteProcessGroupStatus.java        |  217 --
 .../nifi/controller/status/RunStatus.java       |   28 -
 .../controller/status/TransmissionStatus.java   |   23 -
 .../history/ComponentStatusRepository.java      |  167 -
 .../status/history/MetricDescriptor.java        |   75 -
 .../status/history/StatusHistory.java           |   50 -
 .../status/history/StatusSnapshot.java          |   48 -
 .../controller/status/history/ValueMapper.java  |   23 -
 .../controller/status/history/ValueReducer.java |   25 -
 .../org/apache/nifi/events/EventReporter.java   |   27 -
 .../nifi/expression/AttributeExpression.java    |   79 -
 .../expression/AttributeValueDecorator.java     |   29 -
 .../expression/ExpressionLanguageCompiler.java  |   69 -
 .../java/org/apache/nifi/flowfile/FlowFile.java |  109 -
 .../nifi/flowfile/FlowFilePrioritizer.java      |   30 -
 .../java/org/apache/nifi/logging/LogLevel.java  |   27 -
 .../org/apache/nifi/logging/ProcessorLog.java   |   73 -
 .../nifi/processor/AbstractProcessor.java       |   37 -
 .../AbstractSessionFactoryProcessor.java        |  122 -
 .../org/apache/nifi/processor/DataUnit.java     |  248 --
 .../apache/nifi/processor/FlowFileFilter.java   |   91 -
 .../apache/nifi/processor/ProcessContext.java   |  124 -
 .../apache/nifi/processor/ProcessSession.java   |  719 ----
 .../nifi/processor/ProcessSessionFactory.java   |   30 -
 .../org/apache/nifi/processor/Processor.java    |   91 -
 .../ProcessorInitializationContext.java         |   53 -
 .../org/apache/nifi/processor/QueueSize.java    |   49 -
 .../org/apache/nifi/processor/Relationship.java |  127 -
 .../nifi/processor/SchedulingContext.java       |   70 -
 .../annotation/CapabilityDescription.java       |   39 -
 .../nifi/processor/annotation/EventDriven.java  |   49 -
 .../nifi/processor/annotation/OnAdded.java      |   41 -
 .../nifi/processor/annotation/OnRemoved.java    |   42 -
 .../nifi/processor/annotation/OnScheduled.java  |   44 -
 .../nifi/processor/annotation/OnShutdown.java   |   38 -
 .../nifi/processor/annotation/OnStopped.java    |   56 -
 .../processor/annotation/OnUnscheduled.java     |   46 -
 .../processor/annotation/SideEffectFree.java    |   45 -
 .../processor/annotation/SupportsBatching.java  |   51 -
 .../apache/nifi/processor/annotation/Tags.java  |   44 -
 .../processor/annotation/TriggerSerially.java   |   39 -
 .../TriggerWhenAnyDestinationAvailable.java     |   40 -
 .../processor/annotation/TriggerWhenEmpty.java  |   41 -
 .../exception/FlowFileAccessException.java      |   37 -
 .../exception/FlowFileHandlingException.java    |   39 -
 .../exception/MissingFlowFileException.java     |   37 -
 .../processor/exception/ProcessException.java   |   44 -
 .../nifi/processor/io/InputStreamCallback.java  |   38 -
 .../nifi/processor/io/OutputStreamCallback.java |   39 -
 .../nifi/processor/io/StreamCallback.java       |   41 -
 .../nifi/provenance/ProvenanceEventBuilder.java |  308 --
 .../nifi/provenance/ProvenanceEventRecord.java  |  296 --
 .../provenance/ProvenanceEventRepository.java   |  190 -
 .../nifi/provenance/ProvenanceEventType.java    |   92 -
 .../nifi/provenance/ProvenanceReporter.java     |  616 ---
 .../lineage/ComputeLineageResult.java           |   70 -
 .../lineage/ComputeLineageSubmission.java       |   82 -
 .../apache/nifi/provenance/lineage/Lineage.java |   41 -
 .../lineage/LineageComputationType.java         |   27 -
 .../nifi/provenance/lineage/LineageEdge.java    |   26 -
 .../nifi/provenance/lineage/LineageNode.java    |   61 -
 .../provenance/lineage/LineageNodeType.java     |   23 -
 .../lineage/ProvenanceEventLineageNode.java     |   32 -
 .../apache/nifi/provenance/search/Query.java    |   91 -
 .../nifi/provenance/search/QueryResult.java     |   77 -
 .../nifi/provenance/search/QuerySubmission.java |   63 -
 .../nifi/provenance/search/SearchTerm.java      |   24 -
 .../nifi/provenance/search/SearchTerms.java     |   39 -
 .../nifi/provenance/search/SearchableField.java |   62 -
 .../provenance/search/SearchableFieldType.java  |   28 -
 .../nifi/reporting/AbstractReportingTask.java   |   94 -
 .../org/apache/nifi/reporting/Bulletin.java     |  120 -
 .../apache/nifi/reporting/BulletinQuery.java    |  109 -
 .../nifi/reporting/BulletinRepository.java      |   88 -
 .../org/apache/nifi/reporting/EventAccess.java  |   53 -
 .../nifi/reporting/InitializationException.java |   32 -
 .../apache/nifi/reporting/ReportingContext.java |   99 -
 .../ReportingInitializationContext.java         |   80 -
 .../apache/nifi/reporting/ReportingTask.java    |   75 -
 .../org/apache/nifi/reporting/Severity.java     |   24 -
 .../nifi/scheduling/SchedulingStrategy.java     |   86 -
 .../org/apache/nifi/search/SearchContext.java   |   57 -
 .../org/apache/nifi/search/SearchResult.java    |   82 -
 .../java/org/apache/nifi/search/Searchable.java |   27 -
 .../nifi/web/ClusterRequestException.java       |   38 -
 .../nifi/web/InvalidRevisionException.java      |   33 -
 .../org/apache/nifi/web/NiFiWebContext.java     |  121 -
 .../apache/nifi/web/NiFiWebContextConfig.java   |   54 -
 .../nifi/web/ProcessorConfigurationAction.java  |  137 -
 .../java/org/apache/nifi/web/ProcessorInfo.java |  110 -
 .../nifi/web/ResourceNotFoundException.java     |   32 -
 .../main/java/org/apache/nifi/web/Revision.java |  110 -
 .../nifi/components/TestPropertyDescriptor.java |   59 -
 .../org/apache/nifi/processor/TestDataUnit.java |   44 -
 nifi-api/src/test/resources/logback-test.xml    |   27 -
 nifi-bootstrap/pom.xml                          |   29 -
 .../apache/nifi/bootstrap/BootstrapCodec.java   |   92 -
 .../org/apache/nifi/bootstrap/NiFiListener.java |  128 -
 .../java/org/apache/nifi/bootstrap/RunNiFi.java |  941 -----
 .../org/apache/nifi/bootstrap/ShutdownHook.java |   86 -
 .../exception/InvalidCommandException.java      |   37 -
 .../bootstrap/util/LimitingInputStream.java     |  107 -
 nifi-docs/pom.xml                               |  131 -
 .../src/main/asciidoc/administration-guide.adoc |  160 -
 .../src/main/asciidoc/developer-guide.adoc      |   45 -
 .../asciidoc/expression-language-guide.adoc     | 1727 ---------
 .../images/add-processor-with-tag-cloud.png     |  Bin 22925 -> 0 bytes
 .../src/main/asciidoc/images/add-processor.png  |  Bin 31524 -> 0 bytes
 .../src/main/asciidoc/images/addConnect.png     |  Bin 1996 -> 0 bytes
 .../src/main/asciidoc/images/comments-tab.png   |  Bin 6431 -> 0 bytes
 .../src/main/asciidoc/images/components.png     |  Bin 8992 -> 0 bytes
 .../asciidoc/images/connection-settings.png     |  Bin 15214 -> 0 bytes
 .../main/asciidoc/images/create-connection.png  |  Bin 9430 -> 0 bytes
 .../asciidoc/images/edit-property-dropdown.png  |  Bin 14291 -> 0 bytes
 .../asciidoc/images/edit-property-textarea.png  |  Bin 17146 -> 0 bytes
 .../main/asciidoc/images/event-attributes.png   |  Bin 94668 -> 0 bytes
 .../src/main/asciidoc/images/event-content.png  |  Bin 88726 -> 0 bytes
 .../src/main/asciidoc/images/event-details.png  |  Bin 139736 -> 0 bytes
 .../src/main/asciidoc/images/expand-event.png   |  Bin 40476 -> 0 bytes
 .../main/asciidoc/images/expanded-events.png    |  Bin 76082 -> 0 bytes
 .../src/main/asciidoc/images/find-parents.png   |  Bin 35831 -> 0 bytes
 .../src/main/asciidoc/images/iconAlert.png      |  Bin 1396 -> 0 bytes
 .../src/main/asciidoc/images/iconConnection.png |  Bin 1517 -> 0 bytes
 .../src/main/asciidoc/images/iconDelete.png     |  Bin 670 -> 0 bytes
 .../src/main/asciidoc/images/iconDisable.png    |  Bin 764 -> 0 bytes
 nifi-docs/src/main/asciidoc/images/iconEdit.png |  Bin 493 -> 0 bytes
 .../src/main/asciidoc/images/iconEnable.png     |  Bin 667 -> 0 bytes
 .../src/main/asciidoc/images/iconExport.png     |  Bin 453 -> 0 bytes
 .../main/asciidoc/images/iconFlowHistory.png    |  Bin 2463 -> 0 bytes
 .../src/main/asciidoc/images/iconFunnel.png     |  Bin 1223 -> 0 bytes
 nifi-docs/src/main/asciidoc/images/iconInfo.png |  Bin 550 -> 0 bytes
 .../src/main/asciidoc/images/iconInputPort.png  |  Bin 1842 -> 0 bytes
 .../main/asciidoc/images/iconInputPortSmall.png |  Bin 532 -> 0 bytes
 .../src/main/asciidoc/images/iconLabel.png      |  Bin 838 -> 0 bytes
 .../src/main/asciidoc/images/iconLineage.png    |  Bin 2214 -> 0 bytes
 .../main/asciidoc/images/iconNewTemplate.png    |  Bin 557 -> 0 bytes
 .../src/main/asciidoc/images/iconNotSecure.png  |  Bin 221 -> 0 bytes
 .../src/main/asciidoc/images/iconOutputPort.png |  Bin 1658 -> 0 bytes
 .../asciidoc/images/iconOutputPortSmall.png     |  Bin 459 -> 0 bytes
 .../main/asciidoc/images/iconProcessGroup.png   |  Bin 1422 -> 0 bytes
 .../src/main/asciidoc/images/iconProcessor.png  |  Bin 1446 -> 0 bytes
 .../src/main/asciidoc/images/iconProvenance.png |  Bin 2268 -> 0 bytes
 .../asciidoc/images/iconRemoteProcessGroup.png  |  Bin 674 -> 0 bytes
 .../src/main/asciidoc/images/iconResize.png     |  Bin 165 -> 0 bytes
 nifi-docs/src/main/asciidoc/images/iconRun.png  |  Bin 538 -> 0 bytes
 .../src/main/asciidoc/images/iconSecure.png     |  Bin 225 -> 0 bytes
 .../src/main/asciidoc/images/iconSettings.png   |  Bin 2638 -> 0 bytes
 nifi-docs/src/main/asciidoc/images/iconStop.png |  Bin 402 -> 0 bytes
 .../src/main/asciidoc/images/iconSummary.png    |  Bin 272 -> 0 bytes
 .../src/main/asciidoc/images/iconTemplate.png   |  Bin 970 -> 0 bytes
 .../asciidoc/images/iconTransmissionActive.png  |  Bin 1330 -> 0 bytes
 .../images/iconTransmissionInactive.png         |  Bin 1248 -> 0 bytes
 .../src/main/asciidoc/images/iconUsers.png      |  Bin 2272 -> 0 bytes
 .../main/asciidoc/images/iconViewDetails.png    |  Bin 1788 -> 0 bytes
 .../images/instantiate-template-description.png |  Bin 6508 -> 0 bytes
 .../asciidoc/images/instantiate-template.png    |  Bin 3216 -> 0 bytes
 .../main/asciidoc/images/invalid-processor.png  |  Bin 8187 -> 0 bytes
 .../main/asciidoc/images/lineage-flowfile.png   |  Bin 3855 -> 0 bytes
 .../asciidoc/images/lineage-graph-annotated.png |  Bin 170122 -> 0 bytes
 nifi-docs/src/main/asciidoc/images/new-flow.png |  Bin 266913 -> 0 bytes
 .../main/asciidoc/images/nifi-arch-cluster.png  |  Bin 50384 -> 0 bytes
 .../src/main/asciidoc/images/nifi-arch.png      |  Bin 43509 -> 0 bytes
 .../main/asciidoc/images/nifi-navigation.png    |  Bin 339195 -> 0 bytes
 .../asciidoc/images/nifi-toolbar-components.png |  Bin 262314 -> 0 bytes
 .../src/main/asciidoc/images/parent-found.png   |  Bin 42814 -> 0 bytes
 .../asciidoc/images/process-group-anatomy.png   |  Bin 48554 -> 0 bytes
 .../main/asciidoc/images/processor-anatomy.png  |  Bin 37035 -> 0 bytes
 .../images/processor-connection-bubble.png      |  Bin 9936 -> 0 bytes
 .../src/main/asciidoc/images/properties-tab.png |  Bin 11847 -> 0 bytes
 .../asciidoc/images/provenance-annotated.png    |  Bin 530929 -> 0 bytes
 .../main/asciidoc/images/provenance-table.png   |  Bin 466599 -> 0 bytes
 .../asciidoc/images/remote-group-anatomy.png    |  Bin 61383 -> 0 bytes
 .../images/remote-group-ports-dialog.png        |  Bin 21662 -> 0 bytes
 .../images/remote-port-connection-status.png    |  Bin 57332 -> 0 bytes
 .../src/main/asciidoc/images/scheduling-tab.png |  Bin 12288 -> 0 bytes
 .../src/main/asciidoc/images/search-events.png  |  Bin 62626 -> 0 bytes
 .../images/search-receive-event-abc.png         |  Bin 67817 -> 0 bytes
 .../src/main/asciidoc/images/settings-tab.png   |  Bin 16225 -> 0 bytes
 .../src/main/asciidoc/images/simple-flow.png    |  Bin 196808 -> 0 bytes
 .../src/main/asciidoc/images/stats-history.png  |  Bin 33205 -> 0 bytes
 .../src/main/asciidoc/images/status-bar.png     |  Bin 249077 -> 0 bytes
 .../main/asciidoc/images/summary-annotated.png  |  Bin 111956 -> 0 bytes
 .../src/main/asciidoc/images/summary-table.png  |  Bin 62114 -> 0 bytes
 .../main/asciidoc/images/valid-processor.png    |  Bin 8090 -> 0 bytes
 nifi-docs/src/main/asciidoc/overview.adoc       |  296 --
 nifi-docs/src/main/asciidoc/user-guide.adoc     | 1286 -------
 nifi-docs/src/main/assembly/dependencies.xml    |   28 -
 nifi-mock/pom.xml                               |   54 -
 .../MockProvenanceEventRepository.java          |  131 -
 .../apache/nifi/reporting/BulletinFactory.java  |   43 -
 .../org/apache/nifi/reporting/MockBulletin.java |   24 -
 .../util/ControllerServiceConfiguration.java    |   74 -
 .../nifi/util/MockBulletinRepository.java       |   74 -
 .../nifi/util/MockConfigurationContext.java     |   50 -
 ...kControllerServiceInitializationContext.java |   41 -
 .../nifi/util/MockControllerServiceLookup.java  |   76 -
 .../org/apache/nifi/util/MockEventAccess.java   |   70 -
 .../java/org/apache/nifi/util/MockFlowFile.java |  279 --
 .../org/apache/nifi/util/MockFlowFileQueue.java |   85 -
 .../apache/nifi/util/MockProcessContext.java    |  261 --
 .../apache/nifi/util/MockProcessSession.java    | 1010 -----
 .../MockProcessorInitializationContext.java     |   74 -
 .../org/apache/nifi/util/MockProcessorLog.java  |  402 --
 .../org/apache/nifi/util/MockPropertyValue.java |  185 -
 .../nifi/util/MockProvenanceReporter.java       |  202 -
 .../apache/nifi/util/MockReportingContext.java  |  117 -
 .../MockReportingInitializationContext.java     |   81 -
 .../apache/nifi/util/MockSessionFactory.java    |   46 -
 .../apache/nifi/util/MockValidationContext.java |   93 -
 .../org/apache/nifi/util/ReflectionUtils.java   |  149 -
 .../apache/nifi/util/SharedSessionState.java    |   91 -
 .../apache/nifi/util/SingleSessionFactory.java  |   35 -
 .../nifi/util/StandardProcessorTestRunner.java  |  492 ---
 .../java/org/apache/nifi/util/TestRunner.java   |  542 ---
 .../java/org/apache/nifi/util/TestRunners.java  |   37 -
 nifi/assembly/pom.xml                           |  458 +++
 .../assembly/src/main/assembly/dependencies.xml |  140 +
 nifi/commons/data-provenance-utils/.gitignore   |    2 +
 nifi/commons/data-provenance-utils/pom.xml      |   40 +
 .../nifi/provenance/AsyncLineageSubmission.java |   87 +
 .../nifi/provenance/AsyncQuerySubmission.java   |   81 +
 .../nifi/provenance/NamedSearchableField.java   |   95 +
 .../nifi/provenance/SearchableFieldParser.java  |   53 +
 .../nifi/provenance/SearchableFields.java       |   84 +
 .../nifi/provenance/StandardLineageResult.java  |  324 ++
 .../StandardProvenanceEventRecord.java          |  752 ++++
 .../nifi/provenance/StandardQueryResult.java    |  168 +
 .../nifi/provenance/lineage/EdgeNode.java       |   74 +
 .../nifi/provenance/lineage/EventNode.java      |  109 +
 .../provenance/lineage/FlowFileLineage.java     |   76 +
 .../nifi/provenance/lineage/FlowFileNode.java   |   83 +
 nifi/commons/flowfile-packager/pom.xml          |   41 +
 .../org/apache/nifi/util/FlowFilePackager.java  |   28 +
 .../apache/nifi/util/FlowFilePackagerV1.java    |  104 +
 .../apache/nifi/util/FlowFilePackagerV2.java    |  146 +
 .../apache/nifi/util/FlowFilePackagerV3.java    |   93 +
 .../apache/nifi/util/FlowFileUnpackager.java    |   30 +
 .../apache/nifi/util/FlowFileUnpackagerV1.java  |  155 +
 .../apache/nifi/util/FlowFileUnpackagerV2.java  |  143 +
 .../apache/nifi/util/FlowFileUnpackagerV3.java  |  161 +
 .../java/org/apache/nifi/util/Unpackage.java    |  119 +
 .../nifi/util/TestPackageUnpackageV3.java       |   56 +
 nifi/commons/nifi-expression-language/pom.xml   |   56 +
 .../language/antlr/AttributeExpressionLexer.g   |  210 +
 .../language/antlr/AttributeExpressionParser.g  |  139 +
 .../output/AttributeExpressionLexer.tokens      |   88 +
 .../expression/language/EmptyPreparedQuery.java |   62 +
 .../language/InvalidPreparedQuery.java          |   71 +
 .../expression/language/PreparedQuery.java      |   39 +
 .../attribute/expression/language/Query.java    | 1186 ++++++
 .../language/StandardAttributeExpression.java   |   65 +
 .../StandardExpressionLanguageCompiler.java     |   58 +
 .../language/StandardPreparedQuery.java         |   83 +
 .../language/evaluation/BooleanEvaluator.java   |   32 +
 .../language/evaluation/BooleanQueryResult.java |   43 +
 .../language/evaluation/DateEvaluator.java      |   34 +
 .../language/evaluation/DateQueryResult.java    |   45 +
 .../language/evaluation/Evaluator.java          |   32 +
 .../language/evaluation/NumberEvaluator.java    |   33 +
 .../language/evaluation/NumberQueryResult.java  |   43 +
 .../language/evaluation/QueryResult.java        |   26 +
 .../language/evaluation/StringEvaluator.java    |   32 +
 .../language/evaluation/StringQueryResult.java  |   43 +
 .../evaluation/cast/BooleanCastEvaluator.java   |   50 +
 .../evaluation/cast/DateCastEvaluator.java      |  117 +
 .../evaluation/cast/NumberCastEvaluator.java    |   72 +
 .../evaluation/cast/StringCastEvaluator.java    |   49 +
 .../evaluation/functions/AndEvaluator.java      |   60 +
 .../evaluation/functions/AppendEvaluator.java   |   50 +
 .../functions/AttributeEvaluator.java           |   45 +
 .../evaluation/functions/ContainsEvaluator.java |   53 +
 .../functions/DateToNumberEvaluator.java        |   50 +
 .../evaluation/functions/DivideEvaluator.java   |   57 +
 .../evaluation/functions/EndsWithEvaluator.java |   53 +
 .../evaluation/functions/EqualsEvaluator.java   |   89 +
 .../functions/EqualsIgnoreCaseEvaluator.java    |   60 +
 .../evaluation/functions/FindEvaluator.java     |   72 +
 .../evaluation/functions/FormatEvaluator.java   |   61 +
 .../functions/GreaterThanEvaluator.java         |   60 +
 .../functions/GreaterThanOrEqualEvaluator.java  |   60 +
 .../evaluation/functions/HostnameEvaluator.java |   58 +
 .../evaluation/functions/IPEvaluator.java       |   46 +
 .../evaluation/functions/IndexOfEvaluator.java  |   53 +
 .../evaluation/functions/IsEmptyEvaluator.java  |   43 +
 .../evaluation/functions/IsNullEvaluator.java   |   45 +
 .../functions/LastIndexOfEvaluator.java         |   53 +
 .../evaluation/functions/LengthEvaluator.java   |   46 +
 .../evaluation/functions/LessThanEvaluator.java |   60 +
 .../functions/LessThanOrEqualEvaluator.java     |   60 +
 .../evaluation/functions/MatchesEvaluator.java  |   71 +
 .../evaluation/functions/MinusEvaluator.java    |   57 +
 .../evaluation/functions/ModEvaluator.java      |   57 +
 .../evaluation/functions/MultiplyEvaluator.java |   57 +
 .../evaluation/functions/NotEvaluator.java      |   49 +
 .../evaluation/functions/NotNullEvaluator.java  |   45 +
 .../evaluation/functions/NowEvaluator.java      |   39 +
 .../functions/NumberToDateEvaluator.java        |   52 +
 .../functions/OneUpSequenceEvaluator.java       |   41 +
 .../evaluation/functions/OrEvaluator.java       |   60 +
 .../evaluation/functions/PlusEvaluator.java     |   57 +
 .../evaluation/functions/PrependEvaluator.java  |   50 +
 .../functions/ReplaceAllEvaluator.java          |   55 +
 .../functions/ReplaceEmptyEvaluator.java        |   50 +
 .../evaluation/functions/ReplaceEvaluator.java  |   55 +
 .../functions/ReplaceNullEvaluator.java         |   47 +
 .../functions/StartsWithEvaluator.java          |   53 +
 .../functions/StringToDateEvaluator.java        |   65 +
 .../functions/SubstringAfterEvaluator.java      |   59 +
 .../functions/SubstringAfterLastEvaluator.java  |   55 +
 .../functions/SubstringBeforeEvaluator.java     |   58 +
 .../functions/SubstringBeforeLastEvaluator.java |   55 +
 .../functions/SubstringEvaluator.java           |   65 +
 .../evaluation/functions/ToLowerEvaluator.java  |   45 +
 .../evaluation/functions/ToNumberEvaluator.java |   46 +
 .../evaluation/functions/ToRadixEvaluator.java  |   77 +
 .../evaluation/functions/ToStringEvaluator.java |   45 +
 .../evaluation/functions/ToUpperEvaluator.java  |   45 +
 .../evaluation/functions/TrimEvaluator.java     |   45 +
 .../functions/UrlDecodeEvaluator.java           |   55 +
 .../functions/UrlEncodeEvaluator.java           |   55 +
 .../evaluation/functions/UuidEvaluator.java     |   39 +
 .../literals/BooleanLiteralEvaluator.java       |   44 +
 .../literals/NumberLiteralEvaluator.java        |   44 +
 .../literals/StringLiteralEvaluator.java        |   77 +
 .../evaluation/reduce/CountEvaluator.java       |   56 +
 .../evaluation/reduce/JoinEvaluator.java        |   59 +
 .../evaluation/reduce/ReduceEvaluator.java      |   23 +
 .../selection/AllAttributesEvaluator.java       |   68 +
 .../selection/AnyAttributeEvaluator.java        |   68 +
 .../AnyMatchingAttributeEvaluator.java          |   21 +
 .../selection/DelineatedAttributeEvaluator.java |   83 +
 .../evaluation/selection/MappingEvaluator.java  |   61 +
 .../selection/MultiAttributeEvaluator.java      |   24 +
 .../selection/MultiMatchAttributeEvaluator.java |   82 +
 .../selection/MultiNamedAttributeEvaluator.java |   64 +
 .../AttributeExpressionLanguageException.java   |   34 +
 ...ibuteExpressionLanguageParsingException.java |   34 +
 .../exception/IllegalAttributeException.java    |   29 +
 .../expression/language/TestQuery.java          | 1143 ++++++
 .../language/TestStandardPreparedQuery.java     |   92 +
 nifi/commons/nifi-logging-utils/pom.xml         |   36 +
 .../java/org/apache/nifi/logging/NiFiLog.java   |  367 ++
 nifi/commons/nifi-properties/.gitignore         |    3 +
 nifi/commons/nifi-properties/pom.xml            |   29 +
 .../org/apache/nifi/util/NiFiProperties.java    |  876 +++++
 .../java/org/apache/nifi/util/StringUtils.java  |   66 +
 nifi/commons/nifi-security-utils/pom.xml        |   40 +
 .../nifi/security/util/CertificateUtils.java    |  158 +
 .../nifi/security/util/EncryptionMethod.java    |   84 +
 .../apache/nifi/security/util/KeystoreType.java |   26 +
 .../nifi/security/util/SecurityStoreTypes.java  |  144 +
 .../nifi/security/util/SslContextFactory.java   |  180 +
 nifi/commons/nifi-socket-utils/pom.xml          |   60 +
 .../nifi/io/nio/AbstractChannelReader.java      |  166 +
 .../java/org/apache/nifi/io/nio/BufferPool.java |  114 +
 .../apache/nifi/io/nio/ChannelDispatcher.java   |  160 +
 .../org/apache/nifi/io/nio/ChannelListener.java |  228 ++
 .../nifi/io/nio/DatagramChannelReader.java      |   59 +
 .../apache/nifi/io/nio/SocketChannelReader.java |   55 +
 .../io/nio/consumer/AbstractStreamConsumer.java |  132 +
 .../nifi/io/nio/consumer/StreamConsumer.java    |   80 +
 .../io/nio/consumer/StreamConsumerFactory.java  |   27 +
 .../nifi/io/socket/SSLContextFactory.java       |  102 +
 .../io/socket/ServerSocketConfiguration.java    |   83 +
 .../nifi/io/socket/SocketConfiguration.java     |  116 +
 .../apache/nifi/io/socket/SocketListener.java   |  211 +
 .../org/apache/nifi/io/socket/SocketUtils.java  |  169 +
 .../socket/multicast/DiscoverableService.java   |   43 +
 .../multicast/DiscoverableServiceImpl.java      |   78 +
 .../multicast/MulticastConfiguration.java       |   99 +
 .../io/socket/multicast/MulticastListener.java  |  193 +
 .../multicast/MulticastServiceDiscovery.java    |   34 +
 .../multicast/MulticastServicesBroadcaster.java |   33 +
 .../socket/multicast/MulticastTimeToLive.java   |   50 +
 .../io/socket/multicast/MulticastUtils.java     |  109 +
 .../io/socket/multicast/ServiceDiscovery.java   |   31 +
 .../socket/multicast/ServicesBroadcaster.java   |   56 +
 .../apache/nifi/io/nio/example/ServerMain.java  |  141 +
 .../apache/nifi/io/nio/example/TCPClient.java   |   86 +
 .../apache/nifi/io/nio/example/UDPClient.java   |   51 +
 .../io/nio/example/UselessStreamConsumer.java   |   43 +
 .../src/test/resources/log4j.xml                |   36 +
 nifi/commons/nifi-utils/.gitignore              |    8 +
 nifi/commons/nifi-utils/pom.xml                 |   33 +
 .../flowfile/attributes/CoreAttributes.java     |   72 +
 .../attributes/FlowFileAttributeKey.java        |   21 +
 .../nifi/remote/StandardVersionNegotiator.java  |   81 +
 .../apache/nifi/remote/VersionNegotiator.java   |   65 +
 .../TransmissionDisabledException.java          |   25 +
 .../nifi/remote/io/CompressionInputStream.java  |  184 +
 .../nifi/remote/io/CompressionOutputStream.java |  147 +
 .../remote/io/InterruptableInputStream.java     |  117 +
 .../remote/io/InterruptableOutputStream.java    |   81 +
 .../remote/io/socket/BufferStateManager.java    |  111 +
 .../io/socket/SocketChannelInputStream.java     |  157 +
 .../io/socket/SocketChannelOutputStream.java    |  113 +
 .../remote/io/socket/ssl/SSLSocketChannel.java  |  602 +++
 .../socket/ssl/SSLSocketChannelInputStream.java |   62 +
 .../ssl/SSLSocketChannelOutputStream.java       |   53 +
 .../nifi/stream/io/BufferedInputStream.java     |   37 +
 .../nifi/stream/io/BufferedOutputStream.java    |  140 +
 .../nifi/stream/io/ByteArrayInputStream.java    |  250 ++
 .../nifi/stream/io/ByteArrayOutputStream.java   |  250 ++
 .../nifi/stream/io/ByteCountingInputStream.java |  104 +
 .../stream/io/ByteCountingOutputStream.java     |   63 +
 .../apache/nifi/stream/io/DataOutputStream.java |  417 ++
 .../apache/nifi/stream/io/GZIPOutputStream.java |   41 +
 .../stream/io/LeakyBucketStreamThrottler.java   |  331 ++
 .../nifi/stream/io/NonCloseableInputStream.java |   56 +
 .../stream/io/NonCloseableOutputStream.java     |   51 +
 .../apache/nifi/stream/io/NullOutputStream.java |   46 +
 .../apache/nifi/stream/io/StreamThrottler.java  |   33 +
 .../org/apache/nifi/stream/io/StreamUtils.java  |  257 ++
 .../apache/nifi/stream/io/ZipOutputStream.java  |   38 +
 .../exception/BytePatternNotFoundException.java |   28 +
 .../io/util/NonThreadSafeCircularBuffer.java    |   69 +
 .../org/apache/nifi/util/BooleanHolder.java     |   25 +
 .../java/org/apache/nifi/util/FormatUtils.java  |  205 +
 .../org/apache/nifi/util/IntegerHolder.java     |   54 +
 .../java/org/apache/nifi/util/LongHolder.java   |   60 +
 .../apache/nifi/util/NaiveSearchRingBuffer.java |  135 +
 .../java/org/apache/nifi/util/ObjectHolder.java |   39 +
 .../java/org/apache/nifi/util/RingBuffer.java   |  292 ++
 .../java/org/apache/nifi/util/StopWatch.java    |  127 +
 .../main/java/org/apache/nifi/util/Tuple.java   |   83 +
 .../concurrency/DebugDisabledTimedLock.java     |   67 +
 .../util/concurrency/DebugEnabledTimedLock.java |  136 +
 .../util/concurrency/DebuggableTimedLock.java   |   30 +
 .../apache/nifi/util/concurrency/TimedLock.java |   59 +
 .../org/apache/nifi/util/file/FileUtils.java    |  623 +++
 .../file/monitor/CompoundUpdateMonitor.java     |  115 +
 .../util/file/monitor/LastModifiedMonitor.java  |   30 +
 .../nifi/util/file/monitor/MD5SumMonitor.java   |   51 +
 .../file/monitor/SynchronousFileWatcher.java    |  123 +
 .../nifi/util/file/monitor/UpdateMonitor.java   |   25 +
 .../org/apache/nifi/util/search/Search.java     |   57 +
 .../org/apache/nifi/util/search/SearchTerm.java |  141 +
 .../util/search/ahocorasick/AhoCorasick.java    |  155 +
 .../nifi/util/search/ahocorasick/Node.java      |   72 +
 .../util/search/ahocorasick/SearchState.java    |   63 +
 .../nifi/util/timebuffer/EntityAccess.java      |   26 +
 .../nifi/util/timebuffer/LongEntityAccess.java  |   43 +
 .../nifi/util/timebuffer/TimedBuffer.java       |  114 +
 .../nifi/util/timebuffer/TimestampedLong.java   |   35 +
 .../io/TestCompressionInputOutputStreams.java   |  153 +
 .../stream/io/TestLeakyBucketThrottler.java     |  147 +
 .../nifi/util/TestNaiveSearchRingBuffer.java    |   72 +
 .../file/monitor/TestCompoundUpdateMonitor.java |   71 +
 .../monitor/TestSynchronousFileWatcher.java     |   61 +
 .../nifi/util/timebuffer/TestRingBuffer.java    |  182 +
 .../nifi/util/timebuffer/TestTimedBuffer.java   |  106 +
 .../src/test/resources/logback-test.xml         |   32 +
 nifi/commons/nifi-web-utils/pom.xml             |   56 +
 .../org/apache/nifi/web/util/ClientUtils.java   |  132 +
 .../nifi/web/util/ObjectMapperResolver.java     |   48 +
 .../java/org/apache/nifi/web/util/WebUtils.java |  198 +
 nifi/commons/pom.xml                            |   44 +
 nifi/commons/processor-utilities/pom.xml        |   45 +
 .../nifi/processor/util/FlowFileFilters.java    |   65 +
 .../nifi/processor/util/SSLProperties.java      |  226 ++
 .../nifi/processor/util/StandardValidators.java |  544 +++
 .../apache/nifi/processor/TestFormatUtils.java  |   40 +
 .../processor/util/TestStandardValidators.java  |   54 +
 nifi/commons/wali/.gitignore                    |    2 +
 nifi/commons/wali/pom.xml                       |   41 +
 .../org/wali/MinimalLockingWriteAheadLog.java   | 1008 +++++
 .../wali/src/main/java/org/wali/SerDe.java      |  128 +
 .../src/main/java/org/wali/SyncListener.java    |   62 +
 .../wali/src/main/java/org/wali/UpdateType.java |   49 +
 .../java/org/wali/WriteAheadRepository.java     |  122 +
 .../src/test/java/org/wali/DummyRecord.java     |   61 +
 .../test/java/org/wali/DummyRecordSerde.java    |  107 +
 .../wali/TestMinimalLockingWriteAheadLog.java   |  298 ++
 .../execute-script-processors/pom.xml           |   81 +
 .../nifi/processors/script/ExecuteScript.java   |  566 +++
 .../apache/nifi/scripting/ConverterScript.java  |  131 +
 .../nifi/scripting/JRubyScriptFactory.java      |   46 +
 .../nifi/scripting/JavaScriptScriptFactory.java |   56 +
 .../nifi/scripting/JythonScriptFactory.java     |   45 +
 .../nifi/scripting/OutputStreamHandler.java     |   24 +
 .../org/apache/nifi/scripting/ReaderScript.java |   79 +
 .../java/org/apache/nifi/scripting/Script.java  |  303 ++
 .../nifi/scripting/ScriptEngineFactory.java     |  117 +
 .../apache/nifi/scripting/ScriptFactory.java    |  269 ++
 .../org/apache/nifi/scripting/WriterScript.java |   67 +
 .../org.apache.nifi.processor.Processor         |   15 +
 .../index.html                                  |  264 ++
 .../processors/script/TestExecuteScript.java    |  939 +++++
 .../src/test/resources/alwaysFail.js            |   24 +
 .../src/test/resources/alwaysFail.py            |   19 +
 .../src/test/resources/alwaysFail.rb            |   21 +
 .../src/test/resources/ffTest.js                |   28 +
 .../src/test/resources/ffTest.py                |   22 +
 .../src/test/resources/ffTest.rb                |   30 +
 .../src/test/resources/lib/Sub.py               |   18 +
 .../src/test/resources/lib/sub.js               |   22 +
 .../src/test/resources/lib/sub.rb               |   17 +
 .../src/test/resources/loadLocal.js             |   30 +
 .../src/test/resources/loadLocal.py             |   26 +
 .../src/test/resources/loadLocal.rb             |   29 +
 .../src/test/resources/log4j.xml                |   54 +
 .../src/test/resources/optionalValidators.js    |   28 +
 .../src/test/resources/optionalValidators.py    |   22 +
 .../src/test/resources/optionalValidators.rb    |   39 +
 .../src/test/resources/paramTest.js             |   28 +
 .../src/test/resources/paramTest.py             |   26 +
 .../src/test/resources/paramTest.rb             |   31 +
 .../src/test/resources/parseXml.js              |   36 +
 .../src/test/resources/readTest.js              |   30 +
 .../src/test/resources/readTest.py              |   32 +
 .../src/test/resources/readTest.rb              |   30 +
 .../src/test/resources/readWithParams.js        |   32 +
 .../src/test/resources/readWithParams.py        |   32 +
 .../src/test/resources/readWithParams.rb        |   33 +
 .../src/test/resources/routeTest.js             |   41 +
 .../src/test/resources/routeTest.py             |   37 +
 .../src/test/resources/routeTest.rb             |   39 +
 .../src/test/resources/simpleConverter.js       |   45 +
 .../src/test/resources/simpleConverter.py       |   60 +
 .../src/test/resources/simpleConverter.rb       |   42 +
 .../src/test/resources/writeTest.js             |   26 +
 .../src/test/resources/writeTest.py             |   22 +
 .../src/test/resources/writeTest.rb             |   32 +
 .../execute-script-bundle/nar/pom.xml           |   36 +
 nifi/nar-bundles/execute-script-bundle/pom.xml  |   81 +
 .../framework/administration/.gitignore         |    1 +
 .../framework/administration/pom.xml            |  116 +
 .../nifi/admin/AuditDataSourceFactoryBean.java  |  222 ++
 .../org/apache/nifi/admin/RepositoryUtils.java  |   91 +
 .../nifi/admin/UserDataSourceFactoryBean.java   |  247 ++
 .../org/apache/nifi/admin/dao/ActionDAO.java    |   74 +
 .../org/apache/nifi/admin/dao/AuthorityDAO.java |   58 +
 .../org/apache/nifi/admin/dao/DAOFactory.java   |   29 +
 .../nifi/admin/dao/DataAccessException.java     |   39 +
 .../java/org/apache/nifi/admin/dao/UserDAO.java |  127 +
 .../nifi/admin/dao/impl/DAOFactoryImpl.java     |   51 +
 .../nifi/admin/dao/impl/StandardActionDAO.java  | 1056 +++++
 .../admin/dao/impl/StandardAuthorityDAO.java    |  172 +
 .../nifi/admin/dao/impl/StandardUserDAO.java    |  634 +++
 .../admin/service/AccountDisabledException.java |   40 +
 .../admin/service/AccountNotFoundException.java |   40 +
 .../admin/service/AccountPendingException.java  |   41 +
 .../admin/service/AdministrationException.java  |   39 +
 .../apache/nifi/admin/service/AuditService.java |   76 +
 .../apache/nifi/admin/service/UserService.java  |  161 +
 .../service/action/AbstractUserAction.java      |   97 +
 .../admin/service/action/AddActionsAction.java  |   48 +
 .../service/action/AdministrationAction.java    |   38 +
 .../service/action/AuthorizeDownloadAction.java |   54 +
 .../service/action/AuthorizeUserAction.java     |  175 +
 .../admin/service/action/CreateUserAction.java  |   53 +
 .../admin/service/action/DeleteUserAction.java  |   68 +
 .../admin/service/action/DisableUserAction.java |   76 +
 .../service/action/DisableUserGroupAction.java  |   69 +
 .../service/action/FindUserByDnAction.java      |   49 +
 .../service/action/FindUserByIdAction.java      |   49 +
 .../admin/service/action/GetActionAction.java   |   41 +
 .../admin/service/action/GetActionsAction.java  |   48 +
 .../admin/service/action/GetPreviousValues.java |   43 +
 .../service/action/GetUserGroupAction.java      |   50 +
 .../admin/service/action/GetUsersAction.java    |   39 +
 .../service/action/HasPendingUserAccounts.java  |   34 +
 .../action/InvalidateUserAccountAction.java     |   58 +
 .../InvalidateUserGroupAccountsAction.java      |   45 +
 .../service/action/PurgeActionsAction.java      |   51 +
 .../action/RequestUserAccountAction.java        |   67 +
 .../service/action/SeedUserAccountsAction.java  |  164 +
 .../admin/service/action/UngroupUserAction.java |   69 +
 .../service/action/UngroupUserGroupAction.java  |   57 +
 .../admin/service/action/UpdateUserAction.java  |  124 +
 .../UpdateUserAuthoritiesCacheAction.java       |   73 +
 .../service/action/UpdateUserCacheAction.java   |   47 +
 .../service/action/UpdateUserGroupAction.java   |  171 +
 .../service/impl/StandardAuditService.java      |  230 ++
 .../admin/service/impl/StandardUserService.java |  663 ++++
 .../admin/service/transaction/Transaction.java  |   49 +
 .../service/transaction/TransactionBuilder.java |   25 +
 .../transaction/TransactionException.java       |   40 +
 .../transaction/impl/StandardTransaction.java   |   93 +
 .../impl/StandardTransactionBuilder.java        |   57 +
 .../AuthorityProviderFactoryBean.java           |  529 +++
 ...rdAuthorityProviderConfigurationContext.java |   50 +
 ...dAuthorityProviderInitializationContext.java |   42 +
 .../java/org/apache/nifi/history/History.java   |   56 +
 .../org/apache/nifi/history/HistoryQuery.java   |   99 +
 .../org/apache/nifi/history/PreviousValue.java  |   54 +
 .../org/apache/nifi/user/AccountStatus.java     |   47 +
 .../java/org/apache/nifi/user/NiFiUser.java     |  164 +
 .../org/apache/nifi/user/NiFiUserGroup.java     |   45 +
 .../resources/nifi-administration-context.xml   |   62 +
 .../src/main/xsd/authority-providers.xsd        |   49 +
 .../service/action/AuthorizeUserActionTest.java |  433 +++
 .../service/action/CreateUserActionTest.java    |  144 +
 .../service/action/DisableUserActionTest.java   |  171 +
 .../action/InvalidateUserAccountActionTest.java |  131 +
 .../action/RequestUserAccountActionTest.java    |  127 +
 .../action/SeedUserAccountsActionTest.java      |  263 ++
 .../action/SetUserAuthoritiesActionTest.java    |  223 ++
 .../framework/client-dto/.gitignore             |    6 +
 .../framework/client-dto/pom.xml                |   46 +
 .../org/apache/nifi/web/api/dto/AboutDTO.java   |   57 +
 .../org/apache/nifi/web/api/dto/BannerDTO.java  |   57 +
 .../nifi/web/api/dto/BulletinBoardDTO.java      |   63 +
 .../apache/nifi/web/api/dto/BulletinDTO.java    |  161 +
 .../nifi/web/api/dto/BulletinQueryDTO.java      |  113 +
 .../org/apache/nifi/web/api/dto/ClusterDTO.java |   61 +
 .../apache/nifi/web/api/dto/ConnectableDTO.java |  146 +
 .../apache/nifi/web/api/dto/ConnectionDTO.java  |  215 ++
 .../web/api/dto/ControllerConfigurationDTO.java |  158 +
 .../apache/nifi/web/api/dto/ControllerDTO.java  |  262 ++
 .../org/apache/nifi/web/api/dto/CounterDTO.java |   94 +
 .../apache/nifi/web/api/dto/CountersDTO.java    |   61 +
 .../nifi/web/api/dto/DocumentedTypeDTO.java     |   71 +
 .../apache/nifi/web/api/dto/FlowSnippetDTO.java |  141 +
 .../org/apache/nifi/web/api/dto/FunnelDTO.java  |   29 +
 .../org/apache/nifi/web/api/dto/LabelDTO.java   |   94 +
 .../nifi/web/api/dto/NiFiComponentDTO.java      |   95 +
 .../org/apache/nifi/web/api/dto/NodeDTO.java    |  188 +
 .../apache/nifi/web/api/dto/NodeEventDTO.java   |   74 +
 .../web/api/dto/NodeSystemDiagnosticsDTO.java   |   56 +
 .../org/apache/nifi/web/api/dto/PortDTO.java    |  161 +
 .../apache/nifi/web/api/dto/PositionDTO.java    |   65 +
 .../nifi/web/api/dto/PreviousValueDTO.java      |   73 +
 .../nifi/web/api/dto/ProcessGroupDTO.java       |  219 ++
 .../nifi/web/api/dto/ProcessorConfigDTO.java    |  486 +++
 .../apache/nifi/web/api/dto/ProcessorDTO.java   |  181 +
 .../nifi/web/api/dto/ProcessorHistoryDTO.java   |   56 +
 .../nifi/web/api/dto/PropertyHistoryDTO.java    |   43 +
 .../nifi/web/api/dto/RelationshipDTO.java       |   69 +
 .../api/dto/RemoteProcessGroupContentsDTO.java  |   56 +
 .../nifi/web/api/dto/RemoteProcessGroupDTO.java |  279 ++
 .../web/api/dto/RemoteProcessGroupPortDTO.java  |  192 +
 .../apache/nifi/web/api/dto/RevisionDTO.java    |   63 +
 .../org/apache/nifi/web/api/dto/SnippetDTO.java |  239 ++
 .../nifi/web/api/dto/SystemDiagnosticsDTO.java  |  461 +++
 .../apache/nifi/web/api/dto/TemplateDTO.java    |  117 +
 .../org/apache/nifi/web/api/dto/UserDTO.java    |  177 +
 .../apache/nifi/web/api/dto/UserGroupDTO.java   |   84 +
 .../nifi/web/api/dto/action/ActionDTO.java      |  176 +
 .../nifi/web/api/dto/action/HistoryDTO.java     |   74 +
 .../web/api/dto/action/HistoryQueryDTO.java     |  144 +
 .../component/details/ComponentDetailsDTO.java  |   32 +
 .../component/details/ProcessorDetailsDTO.java  |   41 +
 .../details/RemoteProcessGroupDetailsDTO.java   |   41 +
 .../dto/action/details/ActionDetailsDTO.java    |   34 +
 .../dto/action/details/ConfigureDetailsDTO.java |   69 +
 .../dto/action/details/ConnectDetailsDTO.java   |  126 +
 .../api/dto/action/details/MoveDetailsDTO.java  |   83 +
 .../api/dto/action/details/PurgeDetailsDTO.java |   45 +
 .../web/api/dto/provenance/AttributeDTO.java    |   69 +
 .../web/api/dto/provenance/ProvenanceDTO.java   |  165 +
 .../api/dto/provenance/ProvenanceEventDTO.java  |  630 +++
 .../dto/provenance/ProvenanceOptionsDTO.java    |   43 +
 .../dto/provenance/ProvenanceRequestDTO.java    |  118 +
 .../dto/provenance/ProvenanceResultsDTO.java    |  136 +
 .../ProvenanceSearchableFieldDTO.java           |   84 +
 .../api/dto/provenance/lineage/LineageDTO.java  |  161 +
 .../provenance/lineage/LineageRequestDTO.java   |   88 +
 .../provenance/lineage/LineageResultsDTO.java   |   73 +
 .../provenance/lineage/ProvenanceLinkDTO.java   |  101 +
 .../provenance/lineage/ProvenanceNodeDTO.java   |  162 +
 .../dto/search/ComponentSearchResultDTO.java    |   85 +
 .../web/api/dto/search/NodeSearchResultDTO.java |   56 +
 .../web/api/dto/search/SearchResultsDTO.java    |  128 +
 .../dto/search/UserGroupSearchResultDTO.java    |   42 +
 .../web/api/dto/search/UserSearchResultDTO.java |   56 +
 .../dto/status/ClusterConnectionStatusDTO.java  |   89 +
 .../api/dto/status/ClusterPortStatusDTO.java    |   89 +
 .../dto/status/ClusterProcessorStatusDTO.java   |  117 +
 .../ClusterRemoteProcessGroupStatusDTO.java     |   89 +
 .../web/api/dto/status/ClusterStatusDTO.java    |   44 +
 .../api/dto/status/ClusterStatusHistoryDTO.java |   75 +
 .../web/api/dto/status/ConnectionStatusDTO.java |  198 +
 .../web/api/dto/status/ControllerStatusDTO.java |  187 +
 .../api/dto/status/NodeConnectionStatusDTO.java |   57 +
 .../web/api/dto/status/NodePortStatusDTO.java   |   57 +
 .../api/dto/status/NodeProcessorStatusDTO.java  |   57 +
 .../status/NodeRemoteProcessGroupStatusDTO.java |   57 +
 .../nifi/web/api/dto/status/NodeStatusDTO.java  |   57 +
 .../api/dto/status/NodeStatusHistoryDTO.java    |   57 +
 .../nifi/web/api/dto/status/PortStatusDTO.java  |  142 +
 .../api/dto/status/ProcessGroupStatusDTO.java   |  244 ++
 .../web/api/dto/status/ProcessorStatusDTO.java  |  203 +
 .../web/api/dto/status/RemotePortStatusDTO.java |   98 +
 .../dto/status/RemoteProcessGroupStatusDTO.java |  159 +
 .../nifi/web/api/dto/status/StatusDTO.java      |   43 +
 .../web/api/dto/status/StatusDescriptorDTO.java |  101 +
 .../web/api/dto/status/StatusHistoryDTO.java    |   92 +
 .../api/dto/status/StatusHistoryDetailDTO.java  |   56 +
 .../web/api/dto/status/StatusSnapshotDTO.java   |   58 +
 .../nifi/web/api/dto/util/DateTimeAdapter.java  |   47 +
 .../nifi/web/api/dto/util/TimeAdapter.java      |   47 +
 .../nifi/web/api/dto/util/TimestampAdapter.java |   47 +
 .../apache/nifi/web/api/entity/AboutEntity.java |   45 +
 .../nifi/web/api/entity/ActionEntity.java       |   45 +
 .../nifi/web/api/entity/AuthorityEntity.java    |   60 +
 .../nifi/web/api/entity/BannerEntity.java       |   46 +
 .../web/api/entity/BulletinBoardEntity.java     |   45 +
 .../entity/ClusterConnectionStatusEntity.java   |   45 +
 .../nifi/web/api/entity/ClusterEntity.java      |   45 +
 .../web/api/entity/ClusterPortStatusEntity.java |   45 +
 .../entity/ClusterProcessorStatusEntity.java    |   45 +
 .../ClusterRemoteProcessGroupStatusEntity.java  |   45 +
 .../api/entity/ClusterSearchResultsEntity.java  |   46 +
 .../web/api/entity/ClusterStatusEntity.java     |   45 +
 .../api/entity/ClusterStatusHistoryEntity.java  |   45 +
 .../nifi/web/api/entity/ConnectionEntity.java   |   45 +
 .../nifi/web/api/entity/ConnectionsEntity.java  |   47 +
 .../entity/ControllerConfigurationEntity.java   |   45 +
 .../nifi/web/api/entity/ControllerEntity.java   |   45 +
 .../web/api/entity/ControllerStatusEntity.java  |   45 +
 .../nifi/web/api/entity/CounterEntity.java      |   45 +
 .../nifi/web/api/entity/CountersEntity.java     |   46 +
 .../org/apache/nifi/web/api/entity/Entity.java  |   43 +
 .../nifi/web/api/entity/FlowSnippetEntity.java  |   45 +
 .../nifi/web/api/entity/FunnelEntity.java       |   45 +
 .../nifi/web/api/entity/FunnelsEntity.java      |   47 +
 .../nifi/web/api/entity/HistoryEntity.java      |   45 +
 .../nifi/web/api/entity/InputPortEntity.java    |   45 +
 .../nifi/web/api/entity/InputPortsEntity.java   |   47 +
 .../apache/nifi/web/api/entity/LabelEntity.java |   45 +
 .../nifi/web/api/entity/LabelsEntity.java       |   47 +
 .../nifi/web/api/entity/LineageEntity.java      |   45 +
 .../apache/nifi/web/api/entity/NodeEntity.java  |   45 +
 .../nifi/web/api/entity/NodeStatusEntity.java   |   45 +
 .../api/entity/NodeSystemDiagnosticsEntity.java |   45 +
 .../nifi/web/api/entity/OutputPortEntity.java   |   45 +
 .../nifi/web/api/entity/OutputPortsEntity.java  |   47 +
 .../web/api/entity/PrioritizerTypesEntity.java  |   46 +
 .../nifi/web/api/entity/ProcessGroupEntity.java |   45 +
 .../api/entity/ProcessGroupStatusEntity.java    |   45 +
 .../web/api/entity/ProcessGroupsEntity.java     |   46 +
 .../nifi/web/api/entity/ProcessorEntity.java    |   45 +
 .../web/api/entity/ProcessorHistoryEntity.java  |   45 +
 .../web/api/entity/ProcessorTypesEntity.java    |   46 +
 .../nifi/web/api/entity/ProcessorsEntity.java   |   47 +
 .../nifi/web/api/entity/ProvenanceEntity.java   |   40 +
 .../web/api/entity/ProvenanceEventEntity.java   |   45 +
 .../web/api/entity/ProvenanceOptionsEntity.java |   46 +
 .../api/entity/RemoteProcessGroupEntity.java    |   45 +
 .../entity/RemoteProcessGroupPortEntity.java    |   45 +
 .../api/entity/RemoteProcessGroupsEntity.java   |   47 +
 .../web/api/entity/SearchResultsEntity.java     |   46 +
 .../nifi/web/api/entity/SnippetEntity.java      |   45 +
 .../web/api/entity/StatusHistoryEntity.java     |   45 +
 .../web/api/entity/SystemDiagnosticsEntity.java |   45 +
 .../nifi/web/api/entity/TemplateEntity.java     |   45 +
 .../nifi/web/api/entity/TemplatesEntity.java    |   63 +
 .../apache/nifi/web/api/entity/UserEntity.java  |   45 +
 .../nifi/web/api/entity/UserGroupEntity.java    |   45 +
 .../web/api/entity/UserSearchResultsEntity.java |   61 +
 .../apache/nifi/web/api/entity/UsersEntity.java |   64 +
 .../cluster-authorization-provider/.gitignore   |    1 +
 .../cluster-authorization-provider/pom.xml      |   48 +
 .../ClusterManagerAuthorizationProvider.java    |  225 ++
 .../NodeAuthorizationProvider.java              |  389 ++
 .../protocol/message/DoesDnExistMessage.java    |   56 +
 .../protocol/message/GetAuthoritiesMessage.java |   58 +
 .../message/GetGroupForUserMessage.java         |   55 +
 .../protocol/message/ProtocolMessage.java       |   57 +
 .../message/jaxb/JaxbProtocolUtils.java         |   42 +
 .../protocol/message/jaxb/ObjectFactory.java    |   45 +
 ....apache.nifi.authorization.AuthorityProvider |   16 +
 .../framework/cluster-protocol/.gitignore       |    1 +
 .../framework/cluster-protocol/pom.xml          |   69 +
 .../protocol/ClusterManagerProtocolSender.java  |   69 +
 .../cluster/protocol/ConnectionRequest.java     |   44 +
 .../cluster/protocol/ConnectionResponse.java    |  141 +
 .../apache/nifi/cluster/protocol/Heartbeat.java |   68 +
 .../nifi/cluster/protocol/NodeBulletins.java    |   44 +
 .../nifi/cluster/protocol/NodeIdentifier.java   |  172 +
 .../cluster/protocol/NodeProtocolSender.java    |   73 +
 .../nifi/cluster/protocol/ProtocolContext.java  |   39 +
 .../cluster/protocol/ProtocolException.java     |   40 +
 .../nifi/cluster/protocol/ProtocolHandler.java  |   44 +
 .../nifi/cluster/protocol/ProtocolListener.java |   72 +
 .../protocol/ProtocolMessageMarshaller.java     |   38 +
 .../protocol/ProtocolMessageUnmarshaller.java   |   38 +
 .../nifi/cluster/protocol/StandardDataFlow.java |  105 +
 .../UnknownServiceAddressException.java         |   39 +
 .../impl/ClusterManagerProtocolSenderImpl.java  |  245 ++
 .../ClusterManagerProtocolSenderListener.java   |  118 +
 .../protocol/impl/ClusterServiceDiscovery.java  |  181 +
 .../protocol/impl/ClusterServiceLocator.java    |  229 ++
 .../impl/ClusterServicesBroadcaster.java        |  182 +
 .../protocol/impl/CopyingInputStream.java       |   77 +
 .../impl/MulticastProtocolListener.java         |  204 +
 .../protocol/impl/NodeProtocolSenderImpl.java   |  171 +
 .../impl/NodeProtocolSenderListener.java        |  115 +
 .../protocol/impl/SocketProtocolListener.java   |  205 +
 .../protocol/jaxb/JaxbProtocolContext.java      |  148 +
 .../jaxb/message/AdaptedConnectionRequest.java  |   40 +
 .../jaxb/message/AdaptedConnectionResponse.java |  109 +
 .../protocol/jaxb/message/AdaptedCounter.java   |   56 +
 .../protocol/jaxb/message/AdaptedDataFlow.java  |   64 +
 .../protocol/jaxb/message/AdaptedHeartbeat.java |   66 +
 .../jaxb/message/AdaptedNodeBulletins.java      |   50 +
 .../jaxb/message/AdaptedNodeIdentifier.java     |   76 +
 .../jaxb/message/ConnectionRequestAdapter.java  |   41 +
 .../jaxb/message/ConnectionResponseAdapter.java |   55 +
 .../protocol/jaxb/message/DataFlowAdapter.java  |   50 +
 .../protocol/jaxb/message/HeartbeatAdapter.java |   54 +
 .../jaxb/message/JaxbProtocolUtils.java         |   42 +
 .../jaxb/message/NodeBulletinsAdapter.java      |   48 +
 .../jaxb/message/NodeIdentifierAdapter.java     |   51 +
 .../protocol/jaxb/message/ObjectFactory.java    |  104 +
 .../message/ConnectionRequestMessage.java       |   46 +
 .../message/ConnectionResponseMessage.java      |   66 +
 .../ControllerStartupFailureMessage.java        |   49 +
 .../protocol/message/DisconnectMessage.java     |   55 +
 .../protocol/message/ExceptionMessage.java      |   44 +
 .../protocol/message/FlowRequestMessage.java    |   46 +
 .../protocol/message/FlowResponseMessage.java   |   44 +
 .../protocol/message/HeartbeatMessage.java      |   43 +
 .../message/MulticastProtocolMessage.java       |   66 +
 .../protocol/message/NodeBulletinsMessage.java  |   43 +
 .../cluster/protocol/message/PingMessage.java   |   55 +
 .../message/PrimaryRoleAssignmentMessage.java   |   56 +
 .../protocol/message/ProtocolMessage.java       |   61 +
 .../message/ReconnectionFailureMessage.java     |   45 +
 .../message/ReconnectionRequestMessage.java     |   94 +
 .../message/ReconnectionResponseMessage.java    |   32 +
 .../message/ServiceBroadcastMessage.java        |   64 +
 .../MulticastConfigurationFactoryBean.java      |   60 +
 .../ServerSocketConfigurationFactoryBean.java   |   65 +
 .../spring/SocketConfigurationFactoryBean.java  |   66 +
 .../resources/nifi-cluster-protocol-context.xml |  110 +
 .../ClusterManagerProtocolSenderImplTest.java   |  134 +
 .../impl/ClusterServiceDiscoveryTest.java       |  135 +
 .../impl/ClusterServiceLocatorTest.java         |  121 +
 .../impl/ClusterServicesBroadcasterTest.java    |  133 +
 .../impl/MulticastProtocolListenerTest.java     |  171 +
 .../impl/NodeProtocolSenderImplTest.java        |  203 +
 .../impl/testutils/DelayedProtocolHandler.java  |   57 +
 .../testutils/ReflexiveProtocolHandler.java     |   47 +
 .../framework/cluster-web/.gitignore            |    1 +
 .../framework/cluster-web/pom.xml               |   50 +
 .../nifi/cluster/context/ClusterContext.java    |   59 +
 .../cluster/context/ClusterContextImpl.java     |   69 +
 .../context/ClusterContextThreadLocal.java      |   47 +
 .../ClusterAwareOptimisticLockingManager.java   |   96 +
 .../framework/cluster/.gitignore                |    1 +
 .../framework-bundle/framework/cluster/pom.xml  |  132 +
 .../cluster/client/MulticastTestClient.java     |  151 +
 .../org/apache/nifi/cluster/event/Event.java    |  122 +
 .../apache/nifi/cluster/event/EventManager.java |   65 +
 .../cluster/event/impl/EventManagerImpl.java    |  143 +
 .../cluster/firewall/ClusterNodeFirewall.java   |   35 +
 .../impl/FileBasedClusterNodeFirewall.java      |  207 +
 .../nifi/cluster/flow/ClusterDataFlow.java      |   45 +
 .../apache/nifi/cluster/flow/DaoException.java  |   40 +
 .../apache/nifi/cluster/flow/DataFlowDao.java   |   62 +
 .../cluster/flow/DataFlowManagementService.java |  115 +
 .../nifi/cluster/flow/PersistedFlowState.java   |   37 +
 .../nifi/cluster/flow/StaleFlowException.java   |   42 +
 .../nifi/cluster/flow/impl/DataFlowDaoImpl.java |  600 +++
 .../impl/DataFlowManagementServiceImpl.java     |  356 ++
 .../nifi/cluster/manager/ClusterManager.java    |  225 ++
 .../cluster/manager/HttpClusterManager.java     |  169 +
 .../cluster/manager/HttpRequestReplicator.java  |   99 +
 .../cluster/manager/HttpResponseMapper.java     |   42 +
 .../nifi/cluster/manager/NodeResponse.java      |  329 ++
 .../exception/BlockedByFirewallException.java   |   60 +
 .../manager/exception/ClusterException.java     |   40 +
 .../ConnectingNodeMutableRequestException.java  |   41 +
 ...DisconnectedNodeMutableRequestException.java |   41 +
 .../exception/IllegalClusterStateException.java |   41 +
 .../exception/IllegalNodeDeletionException.java |   41 +
 .../IllegalNodeDisconnectionException.java      |   42 +
 .../IllegalNodeReconnectionException.java       |   41 +
 .../IneligiblePrimaryNodeException.java         |   41 +
 .../exception/MutableRequestException.java      |   42 +
 .../exception/NoConnectedNodesException.java    |   41 +
 .../exception/NoResponseFromNodesException.java |   42 +
 .../exception/NodeDisconnectionException.java   |   41 +
 .../exception/NodeReconnectionException.java    |   40 +
 .../PrimaryRoleAssignmentException.java         |   41 +
 .../SafeModeMutableRequestException.java        |   41 +
 .../manager/exception/UnknownNodeException.java |   41 +
 .../exception/UriConstructionException.java     |   42 +
 .../manager/impl/ClusteredEventAccess.java      |  135 +
 .../manager/impl/ClusteredReportingContext.java |  165 +
 .../manager/impl/HttpRequestReplicatorImpl.java |  531 +++
 .../manager/impl/HttpResponseMapperImpl.java    |   85 +
 .../cluster/manager/impl/WebClusterManager.java | 3620 ++++++++++++++++++
 .../java/org/apache/nifi/cluster/node/Node.java |  252 ++
 ...anagerProtocolServiceLocatorFactoryBean.java |  116 +
 ...FileBasedClusterNodeFirewallFactoryBean.java |   58 +
 .../spring/WebClusterManagerFactoryBean.java    |  139 +
 .../reporting/ClusteredReportingTaskNode.java   |   49 +
 .../resources/nifi-cluster-manager-context.xml  |  124 +
 .../event/impl/EventManagerImplTest.java        |  119 +
 .../impl/FileBasedClusterNodeFirewallTest.java  |   98 +
 .../impl/DataFlowManagementServiceImplTest.java |  343 ++
 .../impl/HttpRequestReplicatorImplTest.java     |  368 ++
 .../impl/HttpResponseMapperImplTest.java        |  126 +
 .../manager/impl/TestWebClusterManager.java     |   54 +
 .../cluster/manager/testutils/HttpRequest.java  |  239 ++
 .../cluster/manager/testutils/HttpResponse.java |   93 +
 .../manager/testutils/HttpResponseAction.java   |   60 +
 .../cluster/manager/testutils/HttpServer.java   |  240 ++
 .../ClusterManagerProtocolSenderImplTest.java   |  133 +
 .../impl/ClusterServiceLocatorTest.java         |  119 +
 .../impl/ClusterServicesBroadcasterTest.java    |  131 +
 .../impl/MulticastProtocolListenerTest.java     |  171 +
 .../impl/NodeProtocolSenderImplTest.java        |  201 +
 .../impl/SocketProtocolListenerTest.java        |  132 +
 .../testutils/DelayedProtocolHandler.java       |   57 +
 .../testutils/ReflexiveProtocolHandler.java     |   47 +
 .../cluster/src/test/resources/logback-test.xml |   48 +
 .../apache/nifi/cluster/firewall/impl/empty.txt |    0
 .../apache/nifi/cluster/firewall/impl/ips.txt   |   12 +
 .../framework/core-api/.gitignore               |    1 +
 .../framework-bundle/framework/core-api/pom.xml |   56 +
 .../nifi/cluster/AdaptedNodeInformation.java    |   66 +
 .../nifi/cluster/ClusterNodeInformation.java    |   67 +
 .../org/apache/nifi/cluster/NodeInformant.java  |   22 +
 .../apache/nifi/cluster/NodeInformation.java    |   98 +
 .../nifi/cluster/NodeInformationAdapter.java    |   39 +
 .../apache/nifi/cluster/protocol/DataFlow.java  |   41 +
 .../apache/nifi/connectable/Connectable.java    |  291 ++
 .../nifi/connectable/ConnectableType.java       |   44 +
 .../org/apache/nifi/connectable/Connection.java |   78 +
 .../org/apache/nifi/connectable/Funnel.java     |   24 +
 .../java/org/apache/nifi/connectable/Port.java  |   31 +
 .../org/apache/nifi/connectable/Position.java   |   36 +
 .../java/org/apache/nifi/connectable/Size.java  |   36 +
 .../controller/AbstractConfiguredComponent.java |  280 ++
 .../apache/nifi/controller/AbstractPort.java    |  636 +++
 .../apache/nifi/controller/Availability.java    |   24 +
 .../nifi/controller/ConfiguredComponent.java    |   63 +
 .../nifi/controller/ContentAvailability.java    |   65 +
 .../org/apache/nifi/controller/Counter.java     |   32 +
 .../nifi/controller/EventBasedWorker.java       |   32 +
 .../org/apache/nifi/controller/Heartbeater.java |   22 +
 .../nifi/controller/ProcessScheduler.java       |  146 +
 .../apache/nifi/controller/ProcessorNode.java   |   80 +
 .../nifi/controller/ReportingTaskNode.java      |   56 +
 .../nifi/controller/StandardFlowFileQueue.java  | 1096 ++++++
 .../apache/nifi/controller/StandardFunnel.java  |  541 +++
 .../controller/ValidationContextFactory.java    |   27 +
 .../org/apache/nifi/controller/WorkerQueue.java |   36 +
 .../exception/CommunicationsException.java      |   40 +
 ...ControllerServiceAlreadyExistsException.java |   30 +
 .../ControllerServiceNotFoundException.java     |   51 +
 .../ProcessorInstantiationException.java        |   27 +
 .../exception/ProcessorLifeCycleException.java  |   30 +
 .../org/apache/nifi/controller/label/Label.java |   48 +
 .../ReportingTaskInstantiationException.java    |   31 +
 .../repository/ContentNotFoundException.java    |   48 +
 .../repository/CounterRepository.java           |   34 +
 .../controller/repository/FlowFileEvent.java    |   54 +
 .../repository/FlowFileEventRepository.java     |   50 +
 .../repository/RepositoryStatusReport.java      |   28 +
 .../service/ControllerServiceNode.java          |   40 +
 .../service/ControllerServiceProvider.java      |   47 +
 .../service/ControllerServiceReference.java     |   50 +
 .../org/apache/nifi/events/BulletinFactory.java |   52 +
 .../nifi/events/BulletinProcessingStrategy.java |   27 +
 .../apache/nifi/events/ComponentBulletin.java   |   30 +
 .../org/apache/nifi/events/SystemBulletin.java  |   30 +
 .../org/apache/nifi/groups/ProcessGroup.java    |  723 ++++
 .../apache/nifi/groups/ProcessGroupCounts.java  |   66 +
 .../apache/nifi/groups/RemoteProcessGroup.java  |  255 ++
 .../RemoteProcessGroupPortDescriptor.java       |   92 +
 .../org/apache/nifi/logging/LogMessage.java     |   74 +
 .../org/apache/nifi/logging/LogObserver.java    |   22 +
 .../org/apache/nifi/logging/LogRepository.java  |   67 +
 .../nifi/logging/LogRepositoryFactory.java      |   61 +
 .../java/org/apache/nifi/nar/NarCloseable.java  |   44 +
 .../nifi/nar/NarThreadContextClassLoader.java   |  188 +
 .../main/java/org/apache/nifi/remote/Peer.java  |  107 +
 .../java/org/apache/nifi/remote/PeerStatus.java |   72 +
 .../nifi/remote/PortAuthorizationResult.java    |   25 +
 .../nifi/remote/RemoteAuthorizationState.java   |   27 +
 .../org/apache/nifi/remote/RemoteGroupPort.java |   35 +
 .../org/apache/nifi/remote/RootGroupPort.java   |   78 +
 .../apache/nifi/remote/TransferDirection.java   |   23 +
 .../nifi/remote/VersionedRemoteResource.java    |   24 +
 .../apache/nifi/remote/codec/FlowFileCodec.java |   79 +
 .../remote/exception/BadRequestException.java   |   30 +
 .../remote/exception/HandshakeException.java    |   30 +
 .../exception/NotAuthorizedException.java       |   26 +
 .../exception/PortNotRunningException.java      |   26 +
 .../remote/exception/ProtocolException.java     |   34 +
 .../exception/RequestExpiredException.java      |   26 +
 .../remote/exception/UnknownPortException.java  |   26 +
 .../nifi/remote/protocol/ClientProtocol.java    |   78 +
 .../remote/protocol/CommunicationsInput.java    |   27 +
 .../remote/protocol/CommunicationsOutput.java   |   27 +
 .../remote/protocol/CommunicationsSession.java  |   64 +
 .../nifi/remote/protocol/RequestType.java       |   43 +
 .../nifi/remote/protocol/ServerProtocol.java    |  143 +
 .../framework-bundle/framework/core/.gitignore  |    1 +
 .../framework-bundle/framework/core/pom.xml     |  121 +
 .../apache/nifi/cluster/BulletinsPayload.java   |   95 +
 .../nifi/cluster/ConnectionException.java       |   42 +
 .../nifi/cluster/DisconnectionException.java    |   42 +
 .../apache/nifi/cluster/HeartbeatPayload.java   |  170 +
 .../org/apache/nifi/connectable/LocalPort.java  |  172 +
 .../nifi/connectable/StandardConnection.java    |  336 ++
 .../nifi/controller/EventDrivenWorkerQueue.java |  329 ++
 .../nifi/controller/FileSystemSwapManager.java  |  768 ++++
 .../apache/nifi/controller/FlowController.java  | 3579 +++++++++++++++++
 .../nifi/controller/FlowFromDOMFactory.java     |  418 ++
 .../controller/FlowSerializationException.java  |   48 +
 .../apache/nifi/controller/FlowSerializer.java  |   42 +
 .../FlowSynchronizationException.java           |   47 +
 .../nifi/controller/FlowSynchronizer.java       |   53 +
 .../nifi/controller/FlowUnmarshaller.java       |   78 +
 .../apache/nifi/controller/SnippetManager.java  |   96 +
 .../apache/nifi/controller/StandardCounter.java |  108 +
 .../nifi/controller/StandardFlowSerializer.java |  404 ++
 .../nifi/controller/StandardFlowService.java    |  875 +++++
 .../controller/StandardFlowSynchronizer.java    | 1026 +++++
 .../nifi/controller/StandardProcessorNode.java  | 1243 ++++++
 .../apache/nifi/controller/StandardSnippet.java |  186 +
 .../org/apache/nifi/controller/Template.java    |   37 +
 .../apache/nifi/controller/TemplateManager.java |  507 +++
 .../controller/UninheritableFlowException.java  |   48 +
 .../exception/FlowFileConsumptionException.java |   38 +
 .../ReportingTaskInstantiationException.java    |   31 +
 .../exception/ValidationException.java          |   47 +
 .../nifi/controller/label/StandardLabel.java    |  110 +
 .../reporting/AbstractReportingTaskNode.java    |  111 +
 .../reporting/StandardReportingContext.java     |  132 +
 .../StandardReportingInitializationContext.java |   96 +
 .../reporting/StandardReportingTaskNode.java    |   40 +
 .../repository/BatchingSessionFactory.java      |  247 ++
 .../repository/ConnectionSwapInfo.java          |   58 +
 .../repository/FileSystemRepository.java        | 1351 +++++++
 .../controller/repository/ProcessContext.java   |  291 ++
 .../repository/ProvenanceEventEnricher.java     |   34 +
 .../repository/RepositoryPurgeException.java    |   59 +
 .../repository/RingBufferEventRepository.java   |  312 ++
 .../repository/StandardCounterRepository.java   |  109 +
 .../repository/StandardFlowFileEvent.java       |  237 ++
 .../repository/StandardFlowFileRecord.java      |  312 ++
 .../repository/StandardProcessSession.java      | 2689 +++++++++++++
 .../StandardProcessSessionFactory.java          |   33 +
 .../repository/StandardProvenanceReporter.java  |  431 +++
 .../repository/StandardRepositoryRecord.java    |  196 +
 .../StandardRepositoryStatusReport.java         |   88 +
 .../repository/VolatileContentRepository.java   |  647 ++++
 .../repository/VolatileFlowFileRepository.java  |  109 +
 .../WriteAheadFlowFileRepository.java           |  864 +++++
 .../repository/claim/ContentDirection.java      |   35 +
 .../repository/claim/StandardContentClaim.java  |  154 +
 .../claim/StandardContentClaimManager.java      |  155 +
 .../repository/io/ArrayManagedOutputStream.java |  245 ++
 .../repository/io/ByteCountingInputStream.java  |  100 +
 .../repository/io/ByteCountingOutputStream.java |   64 +
 .../io/DisableOnCloseOutputStream.java          |   73 +
 .../io/FlowFileAccessInputStream.java           |  168 +
 .../io/FlowFileAccessOutputStream.java          |   95 +
 .../repository/io/LimitedInputStream.java       |  107 +
 .../controller/repository/io/LongHolder.java    |   48 +
 .../controller/repository/io/MemoryManager.java |   55 +
 .../repository/io/SyncOnCloseOutputStream.java  |   58 +
 .../scheduling/ConnectableProcessContext.java   |  173 +
 .../scheduling/EventDrivenSchedulingAgent.java  |  346 ++
 .../scheduling/ProcessContextFactory.java       |   51 +
 .../scheduling/QuartzSchedulingAgent.java       |  220 ++
 .../controller/scheduling/ScheduleState.java    |   90 +
 .../controller/scheduling/SchedulingAgent.java  |   45 +
 .../scheduling/StandardProcessScheduler.java    |  569 +++
 .../scheduling/TimerDrivenSchedulingAgent.java  |  139 +
 .../service/ControllerServiceLoader.java        |  156 +
 .../service/StandardConfigurationContext.java   |   64 +
 ...dControllerServiceInitializationContext.java |   64 +
 .../service/StandardControllerServiceNode.java  |  125 +
 .../StandardControllerServiceProvider.java      |  196 +
 .../StandardControllerServiceReference.java     |  100 +
 .../history/StandardMetricDescriptor.java       |  128 +
 .../status/history/StandardStatusHistory.java   |   54 +
 .../status/history/StandardStatusSnapshot.java  |   70 +
 .../status/history/StatusHistoryUtil.java       |  107 +
 .../VolatileComponentStatusRepository.java      |  649 ++++
 .../tasks/ContinuallyRunConnectableTask.java    |   97 +
 .../tasks/ContinuallyRunProcessorTask.java      |  185 +
 .../nifi/controller/tasks/ExpireFlowFiles.java  |  114 +
 .../controller/tasks/ReportingTaskWrapper.java  |   63 +
 .../nifi/diagnostics/DiagnosticUtils.java       |   28 +
 .../nifi/diagnostics/GarbageCollection.java     |   60 +
 .../apache/nifi/diagnostics/StorageUsage.java   |   72 +
 .../nifi/diagnostics/SystemDiagnostics.java     |  212 +
 .../diagnostics/SystemDiagnosticsFactory.java   |  133 +
 .../nifi/encrypt/EncryptionException.java       |   43 +
 .../apache/nifi/encrypt/StringEncryptor.java    |  152 +
 .../java/org/apache/nifi/engine/FlowEngine.java |  135 +
 .../events/NodeBulletinProcessingStrategy.java  |   66 +
 .../nifi/events/VolatileBulletinRepository.java |  240 ++
 .../events/network/CommunicationsFailure.java   |   36 +
 .../nifi/events/network/NetworkTransfer.java    |   48 +
 .../nifi/fingerprint/FingerprintException.java  |   41 +
 .../nifi/fingerprint/FingerprintFactory.java    |  985 +++++
 .../nifi/groups/StandardProcessGroup.java       | 2019 ++++++++++
 .../org/apache/nifi/jaxb/AdaptedBulletin.java   |  100 +
 .../org/apache/nifi/jaxb/AdaptedCounter.java    |   64 +
 .../org/apache/nifi/jaxb/BulletinAdapter.java   |   59 +
 .../org/apache/nifi/jaxb/CounterAdapter.java    |   51 +
 .../org/apache/nifi/lifecycle/LifeCycle.java    |   54 +
 .../nifi/lifecycle/LifeCycleException.java      |   48 +
 .../nifi/lifecycle/LifeCycleStartException.java |   49 +
 .../nifi/lifecycle/LifeCycleStopException.java  |   48 +
 .../nifi/logging/ProcessorLogObserver.java      |   48 +
 .../repository/StandardLogRepository.java       |  173 +
 .../nifi/persistence/FlowConfigurationDAO.java  |  135 +
 .../StandardSnippetDeserializer.java            |   42 +
 .../persistence/StandardSnippetSerializer.java  |   47 +
 .../StandardXMLFlowConfigurationDAO.java        |  345 ++
 .../nifi/persistence/TemplateDeserializer.java  |   42 +
 .../nifi/persistence/TemplateSerializer.java    |   47 +
 .../nifi/processor/SimpleProcessLogger.java     |  307 ++
 .../nifi/processor/StandardProcessContext.java  |  145 +
 .../StandardProcessorInitializationContext.java |   49 +
 .../nifi/processor/StandardPropertyValue.java   |  163 +
 .../processor/StandardSchedulingContext.java    |  107 +
 .../processor/StandardValidationContext.java    |   97 +
 .../StandardValidationContextFactory.java       |   39 +
 .../nifi/remote/StandardRemoteProcessGroup.java | 1621 ++++++++
 ...tandardRemoteProcessGroupPortDescriptor.java |  148 +
 .../org/apache/nifi/services/FlowService.java   |  139 +
 .../nifi/spring/FlowControllerFactoryBean.java  |  103 +
 .../spring/RingBufferEventRepositoryBean.java   |   45 +
 .../spring/StandardFlowServiceFactoryBean.java  |   98 +
 .../apache/nifi/util/ComponentStatusReport.java |  139 +
 .../java/org/apache/nifi/util/Connectables.java |   56 +
 .../java/org/apache/nifi/util/DomUtils.java     |   79 +
 .../org/apache/nifi/util/ReflectionUtils.java   |  157 +
 .../java/org/apache/nifi/util/SnippetUtils.java |  241 ++
 .../ControllerServiceConfiguration.xsd          |   61 +
 .../src/main/resources/FlowConfiguration.xsd    |  335 ++
 .../resources/ReportingTaskConfiguration.xsd    |   87 +
 .../core/src/main/resources/nifi-context.xml    |   49 +
 .../nifi/cluster/HeartbeatPayloadTest.java      |  121 +
 .../controller/StandardFlowServiceTest.java     |  300 ++
 .../controller/TestFileSystemSwapManager.java   |  104 +
 .../repository/TestFileSystemRepository.java    |  352 ++
 .../TestRingBufferEventRepository.java          |  138 +
 .../repository/TestStandardProcessSession.java  | 1177 ++++++
 .../TestStandardProvenanceReporter.java         |   65 +
 .../TestVolatileContentRepository.java          |  183 +
 .../TestWriteAheadFlowFileRepository.java       |  135 +
 .../repository/io/TestLimitedOutputStream.java  |   76 +
 .../controller/repository/util/DiskUtils.java   |   77 +
 .../fingerprint/FingerprintFactoryTest.java     |   60 +
 .../processor/TestStandardPropertyValue.java    |  167 +
 .../processors/DataGeneratorTestProcessor.java  |   85 +
 .../StubAttributeLoggerProcessor.java           |  111 +
 .../StubTerminationFileProcessor.java           |   40 +
 .../core/src/test/resources/conf/0bytes.xml     |    0
 .../test/resources/conf/all-flow-corrupt.xml    |  201 +
 .../resources/conf/all-flow-inheritable.xml     |  196 +
 .../resources/conf/all-flow-uninheritable.xml   |  202 +
 .../core/src/test/resources/conf/all-flow.xml   |  198 +
 .../src/test/resources/conf/nifi.properties     |  127 +
 .../test/resources/conf/only-termination.xml    |   37 +
 .../src/test/resources/conf/remote-flow.xml     |  145 +
 .../src/test/resources/conf/standard-flow.xml   |  196 +
 .../core/src/test/resources/conf/taskConfig.xml |   17 +
 .../test/resources/conf/termination-only.xml    |    0
 .../framework/core/src/test/resources/hello.txt |    1 +
 .../core/src/test/resources/logback-test.xml    |   36 +
 .../test/resources/nifi-with-remote.properties  |  127 +
 .../core/src/test/resources/nifi.properties     |  127 +
 .../test/resources/nifi/fingerprint/flow1a.xml  |  160 +
 .../test/resources/nifi/fingerprint/flow1b.xml  |  160 +
 .../test/resources/nifi/fingerprint/flow2.xml   |  160 +
 .../core/src/test/resources/old-swap-file.swap  |  Bin 0 -> 1730054 bytes
 .../file-authorization-provider/pom.xml         |   81 +
 .../FileAuthorizationProvider.java              |  584 +++
 ....apache.nifi.authorization.AuthorityProvider |   15 +
 .../src/main/xsd/users.xsd                      |   64 +
 .../FileAuthorizationProviderTest.java          |  127 +
 .../framework-bundle/framework/nar/.gitignore   |    1 +
 .../framework-bundle/framework/nar/pom.xml      |   36 +
 .../org/apache/nifi/nar/ExtensionManager.java   |  188 +
 .../org/apache/nifi/nar/ExtensionMapping.java   |   73 +
 .../org/apache/nifi/nar/NarClassLoader.java     |  227 ++
 .../org/apache/nifi/nar/NarClassLoaders.java    |  301 ++
 .../java/org/apache/nifi/nar/NarUnpacker.java   |  351 ++
 .../java/org/apache/nifi/util/FileUtils.java    |  187 +
 .../framework-bundle/framework/pom.xml          |   51 +
 .../framework/resources/pom.xml                 |   52 +
 .../src/main/assembly/dependencies.xml          |   36 +
 .../src/main/resources/bin/dump-nifi.bat        |   33 +
 .../src/main/resources/bin/nifi-status.bat      |   33 +
 .../resources/src/main/resources/bin/nifi.sh    |  181 +
 .../src/main/resources/bin/run-nifi.bat         |   33 +
 .../src/main/resources/bin/start-nifi.bat       |   33 +
 .../src/main/resources/bin/stop-nifi.bat        |   33 +
 .../main/resources/conf/authority-providers.xml |   43 +
 .../main/resources/conf/authorized-users.xml    |   57 +
 .../src/main/resources/conf/bootstrap.conf      |   57 +
 .../main/resources/conf/controller-services.xml |   18 +
 .../src/main/resources/conf/logback.xml         |  105 +
 .../src/main/resources/conf/nifi.properties     |  159 +
 .../src/main/resources/conf/reporting-tasks.xml |   49 +
 .../framework/runtime/.gitignore                |    2 +
 .../framework-bundle/framework/runtime/pom.xml  |   42 +
 .../java/org/apache/nifi/BootstrapListener.java |  378 ++
 .../src/main/java/org/apache/nifi/NiFi.java     |  223 ++
 .../main/java/org/apache/nifi/NiFiServer.java   |   31 +
 .../apache/nifi/util/LimitingInputStream.java   |  107 +
 .../framework/security/.gitignore               |    1 +
 .../framework-bundle/framework/security/pom.xml |   42 +
 .../util/SslContextCreationException.java       |   40 +
 .../security/util/SslContextFactory.java        |  119 +
 .../framework/security/util/SslException.java   |   40 +
 .../security/util/SslServerSocketFactory.java   |   81 +
 ...SslServerSocketFactoryCreationException.java |   40 +
 .../security/util/SslSocketFactory.java         |   91 +
 .../util/SslSocketFactoryCreationException.java |   40 +
 .../security/util/SslContextFactoryTest.java    |   69 +
 .../src/test/resources/localhost-ks.jks         |  Bin 0 -> 1298 bytes
 .../src/test/resources/localhost-ts.jks         |  Bin 0 -> 891 bytes
 .../src/test/resources/log4j.properties         |   21 +
 .../framework/site-to-site/.gitignore           |    1 +
 .../framework/site-to-site/pom.xml              |   74 +
 .../util/RemoteProcessGroupUtils.java           |  216 ++
 .../remote/AbstractCommunicationsSession.java   |   54 +
 .../nifi/remote/RemoteResourceFactory.java      |  202 +
 .../nifi/remote/RemoteResourceManager.java      |  227 ++
 .../apache/nifi/remote/RemoteSiteListener.java  |   33 +
 .../nifi/remote/SocketRemoteSiteListener.java   |  340 ++
 .../nifi/remote/StandardRemoteGroupPort.java    |  764 ++++
 .../nifi/remote/StandardRootGroupPort.java      |  629 +++
 .../remote/codec/StandardFlowFileCodec.java     |  169 +
 .../exception/UnsupportedCodecException.java    |   31 +
 .../SocketChannelCommunicationsSession.java     |   90 +
 .../remote/io/socket/SocketChannelInput.java    |   66 +
 .../remote/io/socket/SocketChannelOutput.java   |   58 +
 .../SSLSocketChannelCommunicationsSession.java  |   93 +
 .../io/socket/ssl/SSLSocketChannelInput.java    |   50 +
 .../io/socket/ssl/SSLSocketChannelOutput.java   |   44 +
 .../socket/ClusterManagerServerProtocol.java    |  208 +
 .../protocol/socket/HandshakeProperty.java      |   23 +
 .../nifi/remote/protocol/socket/Response.java   |   51 +
 .../remote/protocol/socket/ResponseCode.java    |  152 +
 .../protocol/socket/SocketClientProtocol.java   |  510 +++
 .../socket/SocketFlowFileServerProtocol.java    |  581 +++
 .../org.apache.nifi.remote.codec.FlowFileCodec  |   15 +
 ...g.apache.nifi.remote.protocol.ClientProtocol |   15 +
 ...g.apache.nifi.remote.protocol.ServerProtocol |   16 +
 .../remote/TestStandardRemoteGroupPort.java     |   97 +
 .../remote/TestStandardSiteToSiteProtocol.java  |  116 +
 .../io/socket/TestSocketChannelStreams.java     |  231 ++
 .../io/socket/ssl/TestSSLSocketChannel.java     |  381 ++
 .../test/resources/dummy-certs/localhost-ks.jks |  Bin 0 -> 1298 bytes
 .../test/resources/dummy-certs/localhost-ts.jks |  Bin 0 -> 891 bytes
 .../src/test/resources/logback-test.xml         |   30 +
 .../src/test/resources/nifi.properties          |  111 +
 .../framework/user-actions/.gitignore           |    1 +
 .../framework/user-actions/pom.xml              |   27 +
 .../java/org/apache/nifi/action/Action.java     |  121 +
 .../java/org/apache/nifi/action/Component.java  |   31 +
 .../java/org/apache/nifi/action/Operation.java  |   36 +
 .../component/details/ComponentDetails.java     |   26 +
 .../component/details/ProcessorDetails.java     |   34 +
 .../details/RemoteProcessGroupDetails.java      |   34 +
 .../nifi/action/details/ActionDetails.java      |   26 +
 .../nifi/action/details/ConfigureDetails.java   |   52 +
 .../nifi/action/details/ConnectDetails.java     |   90 +
 .../apache/nifi/action/details/MoveDetails.java |   61 +
 .../nifi/action/details/PurgeDetails.java       |   41 +
 .../web/custom-ui-utilities/.gitignore          |    1 +
 .../framework/web/custom-ui-utilities/pom.xml   |   36 +
 .../web/HttpServletRequestContextConfig.java    |  118 +
 .../framework/web/nifi-jetty/.gitignore         |    1 +
 .../framework/web/nifi-jetty/pom.xml            |  166 +
 .../src/main/assembly/distribution.xml          |   42 +
 .../org/apache/nifi/web/server/JettyServer.java |  686 ++++
 .../server/ServerConfigurationException.java    |   36 +
 .../org/apache/nifi/web/webdefault.xml          |  556 +++
 .../framework/web/nifi-web-api/.gitignore       |    1 +
 .../framework/web/nifi-web-api/pom.xml          |  305 ++
 .../src/main/enunciate/enunciate.xml            |   36 +
 .../src/main/enunciate/images/home.png          |  Bin 0 -> 144 bytes
 .../src/main/enunciate/override.css             |  178 +
 .../org/apache/nifi/aop/MethodProfiler.java     |   45 +
 .../apache/nifi/audit/ControllerAuditor.java    |  257 ++
 .../org/apache/nifi/audit/FunnelAuditor.java    |  135 +
 .../java/org/apache/nifi/audit/NiFiAuditor.java |  119 +
 .../java/org/apache/nifi/audit/PortAuditor.java |  334 ++
 .../apache/nifi/audit/ProcessGroupAuditor.java  |  263 ++
 .../org/apache/nifi/audit/ProcessorAuditor.java |  408 ++
 .../apache/nifi/audit/RelationshipAuditor.java  |  412 ++
 .../nifi/audit/RemoteProcessGroupAuditor.java   |  376 ++
 .../org/apache/nifi/audit/SnippetAuditor.java   |  519 +++
 .../nifi/web/ClusterRequestException.java       |   38 +
 .../IllegalClusterResourceRequestException.java |   33 +
 .../org/apache/nifi/web/NiFiCoreException.java  |   32 +
 .../org/apache/nifi/web/NiFiServiceFacade.java  | 1263 ++++++
 .../apache/nifi/web/NiFiServiceFacadeLock.java  |  177 +
 .../nifi/web/StandardNiFiServiceFacade.java     | 2877 ++++++++++++++
 .../apache/nifi/web/StandardNiFiWebContext.java |  336 ++
 .../nifi/web/api/ApplicationResource.java       |  399 ++
 .../nifi/web/api/BulletinBoardResource.java     |  116 +
 .../apache/nifi/web/api/ClusterResource.java    |  724 ++++
 .../apache/nifi/web/api/ConnectionResource.java |  771 ++++
 .../apache/nifi/web/api/ControllerResource.java |  803 ++++
 .../org/apache/nifi/web/api/FunnelResource.java |  493 +++
 .../apache/nifi/web/api/HistoryResource.java    |  271 ++
 .../apache/nifi/web/api/InputPortResource.java  |  531 +++
 .../org/apache/nifi/web/api/LabelResource.java  |  547 +++
 .../org/apache/nifi/web/api/NodeResource.java   |  302 ++
 .../apache/nifi/web/api/OutputPortResource.java |  532 +++
 .../nifi/web/api/ProcessGroupResource.java      | 1102 ++++++
 .../apache/nifi/web/api/ProcessorResource.java  |  693 ++++
 .../apache/nifi/web/api/ProvenanceResource.java | 1068 ++++++
 .../web/api/RemoteProcessGroupResource.java     |  912 +++++
 .../apache/nifi/web/api/SnippetResource.java    |  601 +++
 .../nifi/web/api/SystemDiagnosticsResource.java |   81 +
 .../apache/nifi/web/api/TemplateResource.java   |  400 ++
 .../apache/nifi/web/api/UserGroupResource.java  |  383 ++
 .../org/apache/nifi/web/api/UserResource.java   |  469 +++
 .../api/config/AccessDeniedExceptionMapper.java |   54 +
 .../config/AccountNotFoundExceptionMapper.java  |   47 +
 .../config/AdministrationExceptionMapper.java   |   48 +
 ...ationCredentialsNotFoundExceptionMapper.java |   48 +
 .../web/api/config/ClusterExceptionMapper.java  |   47 +
 ...ectingNodeMutableRequestExceptionMapper.java |   49 +
 ...nectedNodeMutableRequestExceptionMapper.java |   49 +
 .../config/IllegalArgumentExceptionMapper.java  |   46 +
 ...alClusterResourceRequestExceptionMapper.java |   47 +
 .../IllegalClusterStateExceptionMapper.java     |   47 +
 .../IllegalNodeDeletionExceptionMapper.java     |   46 +
 ...IllegalNodeDisconnectionExceptionMapper.java |   46 +
 .../IllegalNodeReconnectionExceptionMapper.java |   46 +
 .../api/config/IllegalStateExceptionMapper.java |   46 +
 .../IneligiblePrimaryNodeExceptionMapper.java   |   47 +
 .../config/InvalidRevisionExceptionMapper.java  |   47 +
 .../api/config/JsonMappingExceptionMapper.java  |   46 +
 .../api/config/JsonParseExceptionMapper.java    |   46 +
 .../config/MutableRequestExceptionMapper.java   |   48 +
 .../web/api/config/NiFiCoreExceptionMapper.java |   47 +
 .../config/NoConnectedNodesExceptionMapper.java |   47 +
 .../NoResponseFromNodesExceptionMapper.java     |   49 +
 .../NodeDisconnectionExceptionMapper.java       |   47 +
 .../config/NodeReconnectionExceptionMapper.java |   47 +
 .../PrimaryRoleAssignmentExceptionMapper.java   |   47 +
 .../config/ResourceNotFoundExceptionMapper.java |   48 +
 .../SafeModeMutableRequestExceptionMapper.java  |   48 +
 .../nifi/web/api/config/ThrowableMapper.java    |   46 +
 .../api/config/UnknownNodeExceptionMapper.java  |   47 +
 .../api/config/ValidationExceptionMapper.java   |   49 +
 .../config/WebApplicationExceptionMapper.java   |   63 +
 .../org/apache/nifi/web/api/dto/DtoFactory.java | 2142 +++++++++++
 .../org/apache/nifi/web/api/package-info.java   |   58 +
 .../request/BulletinBoardPatternParameter.java  |   47 +
 .../nifi/web/api/request/ClientIdParameter.java |   45 +
 .../api/request/ConnectableTypeParameter.java   |   41 +
 .../nifi/web/api/request/DateTimeParameter.java |   63 +
 .../nifi/web/api/request/DoubleParameter.java   |   40 +
 .../nifi/web/api/request/IntegerParameter.java  |   40 +
 .../nifi/web/api/request/LongParameter.java     |   40 +
 .../ApplicationStartupContextListener.java      |  157 +
 .../nifi/web/controller/ControllerFacade.java   | 1392 +++++++
 .../web/controller/StandardSearchContext.java   |   64 +
 .../org/apache/nifi/web/dao/ConnectionDAO.java  |  112 +
 .../java/org/apache/nifi/web/dao/FunnelDAO.java |   88 +
 .../java/org/apache/nifi/web/dao/LabelDAO.java  |   79 +
 .../java/org/apache/nifi/web/dao/PortDAO.java   |   96 +
 .../apache/nifi/web/dao/ProcessGroupDAO.java    |   89 +
 .../org/apache/nifi/web/dao/ProcessorDAO.java   |   97 +
 .../nifi/web/dao/RemoteProcessGroupDAO.java     |  135 +
 .../org/apache/nifi/web/dao/SnippetDAO.java     |   92 +
 .../org/apache/nifi/web/dao/TemplateDAO.java    |   77 +
 .../apache/nifi/web/dao/impl/ComponentDAO.java  |   72 +
 .../web/dao/impl/StandardConnectionDAO.java     |  519 +++
 .../nifi/web/dao/impl/StandardFunnelDAO.java    |  180 +
 .../nifi/web/dao/impl/StandardInputPortDAO.java |  309 ++
 .../nifi/web/dao/impl/StandardLabelDAO.java     |  184 +
 .../web/dao/impl/StandardOutputPortDAO.java     |  309 ++
 .../web/dao/impl/StandardProcessGroupDAO.java   |  171 +
 .../nifi/web/dao/impl/StandardProcessorDAO.java |  503 +++
 .../dao/impl/StandardRemoteProcessGroupDAO.java |  402 ++
 .../nifi/web/dao/impl/StandardSnippetDAO.java   |  344 ++
 .../nifi/web/dao/impl/StandardTemplateDAO.java  |  141 +
 .../nifi/web/filter/NodeRequestFilter.java      |  130 +
 .../apache/nifi/web/filter/RequestLogger.java   |   78 +
 .../nifi/web/filter/ThreadLocalFilter.java      |   54 +
 .../org/apache/nifi/web/filter/TimerFilter.java |   70 +
 .../nifi/web/util/ClientResponseUtils.java      |   48 +
 .../nifi/web/util/DownloadableContent.java      |   47 +
 .../org/apache/nifi/web/util/SnippetUtils.java  |  426 +++
 .../src/main/resources/nifi-web-api-context.xml |  303 ++
 .../src/main/webapp/WEB-INF/web.xml             |  105 +
 .../apache/nifi/integration/NiFiWebApiTest.java |  274 ++
 .../accesscontrol/AdminAccessControlTest.java   | 1090 ++++++
 .../accesscontrol/DfmAccessControlTest.java     | 1410 +++++++
 .../ReadOnlyAccessControlTest.java              | 1064 +++++
 .../util/NiFiTestAuthorizationProvider.java     |  179 +
 .../nifi/integration/util/NiFiTestServer.java   |  235 ++
 .../nifi/integration/util/NiFiTestUser.java     |  257 ++
 .../integration/util/SourceTestProcessor.java   |   49 +
 .../util/TerminationTestProcessor.java          |   44 +
 .../nifi/integration/util/TestPrioritizer.java  |   31 +
 ....apache.nifi.authorization.AuthorityProvider |   15 +
 ...org.apache.nifi.flowfile.FlowFilePrioritizer |   15 +
 .../org.apache.nifi.processor.Processor         |   16 +
 .../access-control/authority-providers.xml      |   24 +
 .../access-control/controller-services.xml      |   18 +
 .../resources/access-control/localhost-ks.jks   |  Bin 0 -> 2246 bytes
 .../resources/access-control/localhost-ts.jks   |  Bin 0 -> 957 bytes
 .../resources/access-control/nifi.properties    |  136 +
 .../access-control/reporting-tasks.xml          |   17 +
 .../src/test/resources/logback-test.xml         |   36 +
 .../framework/web/nifi-web-docs/.gitignore      |    1 +
 .../framework/web/nifi-web-docs/pom.xml         |   57 +
 .../nifi/web/docs/DocumentationController.java  |   91 +
 .../main/webapp/WEB-INF/jsp/documentation.jsp   |  130 +
 .../WEB-INF/jsp/no-documentation-found.jsp      |   31 +
 .../src/main/webapp/WEB-INF/web.xml             |   33 +
 .../src/main/webapp/css/component-usage.css     |  127 +
 .../nifi-web-docs/src/main/webapp/css/main.css  |  214 ++
 .../src/main/webapp/images/bgBannerFoot.png     |  Bin 0 -> 189 bytes
 .../src/main/webapp/images/bgHeader.png         |  Bin 0 -> 1697 bytes
 .../src/main/webapp/images/bgTableHeader.png    |  Bin 0 -> 232 bytes
 .../src/main/webapp/js/application.js           |  282 ++
 .../framework/web/nifi-web-error/.gitignore     |    1 +
 .../framework/web/nifi-web-error/pom.xml        |   48 +
 .../apache/nifi/web/filter/CatchAllFilter.java  |   45 +
 .../src/main/webapp/WEB-INF/web.xml             |   26 +
 .../nifi-web-error/src/main/webapp/index.jsp    |   37 +
 .../framework/web/nifi-web-ui/.gitignore        |    1 +
 .../framework/web/nifi-web-ui/pom.xml           |  648 ++++
 .../apache/nifi/web/filter/IeEdgeHeader.java    |   57 +
 .../org/apache/nifi/web/servlet/ConvertSvg.java |  117 +
 .../filters/bulletin-board-min.properties       |   18 +
 .../resources/filters/bulletin-board.properties |   25 +
 .../resources/filters/canvas-min.properties     |   18 +
 .../main/resources/filters/canvas.properties    |   69 +
 .../resources/filters/cluster-min.properties    |   18 +
 .../main/resources/filters/cluster.properties   |   26 +
 .../resources/filters/counters-min.properties   |   18 +
 .../main/resources/filters/counters.properties  |   26 +
 .../resources/filters/history-min.properties    |   18 +
 .../main/resources/filters/history.properties   |   27 +
 .../resources/filters/provenance-min.properties |   18 +
 .../resources/filters/provenance.properties     |   26 +
 .../resources/filters/summary-min.properties    |   18 +
 .../main/resources/filters/summary.properties   |   32 +
 .../resources/filters/templates-min.properties  |   18 +
 .../main/resources/filters/templates.properties |   26 +
 .../main/resources/filters/users-min.properties |   18 +
 .../src/main/resources/filters/users.properties |   26 +
 .../webapp/WEB-INF/pages/bulletin-board.jsp     |   50 +
 .../src/main/webapp/WEB-INF/pages/canvas.jsp    |  118 +
 .../src/main/webapp/WEB-INF/pages/cluster.jsp   |   62 +
 .../src/main/webapp/WEB-INF/pages/counters.jsp  |   61 +
 .../src/main/webapp/WEB-INF/pages/history.jsp   |   65 +
 .../main/webapp/WEB-INF/pages/message-page.jsp  |   34 +
 .../main/webapp/WEB-INF/pages/provenance.jsp    |   68 +
 .../src/main/webapp/WEB-INF/pages/summary.jsp   |   80 +
 .../src/main/webapp/WEB-INF/pages/templates.jsp |   63 +
 .../src/main/webapp/WEB-INF/pages/users.jsp     |   71 +
 .../webapp/WEB-INF/partials/banners-main.jsp    |   19 +
 .../webapp/WEB-INF/partials/banners-utility.jsp |   19 +
 .../bulletin-board/bulletin-board-content.jsp   |   40 +
 .../WEB-INF/partials/canvas/about-dialog.jsp    |   29 +
 .../WEB-INF/partials/canvas/canvas-header.jsp   |   56 +
 .../canvas/connection-configuration.jsp         |  180 +
 .../partials/canvas/connections-dialog.jsp      |   35 +
 .../partials/canvas/fill-color-dialog.jsp       |   42 +
 .../WEB-INF/partials/canvas/flow-status.jsp     |   55 +
 .../canvas/instantiate-template-dialog.jsp      |   22 +
 .../partials/canvas/label-configuration.jsp     |   38 +
 .../WEB-INF/partials/canvas/navigation.jsp      |   57 +
 .../WEB-INF/partials/canvas/new-port-dialog.jsp |   27 +
 .../canvas/new-process-group-dialog.jsp         |   27 +
 .../partials/canvas/new-processor-dialog.jsp    |   50 +
 .../canvas/new-processor-property-dialog.jsp    |   34 +
 .../canvas/new-remote-process-group-dialog.jsp  |   27 +
 .../partials/canvas/new-template-dialog.jsp     |   33 +
 .../partials/canvas/port-configuration.jsp      |   54 +
 .../WEB-INF/partials/canvas/port-details.jsp    |   39 +
 .../canvas/process-group-configuration.jsp      |   34 +
 .../partials/canvas/process-group-details.jsp   |   33 +
 .../partials/canvas/processor-configuration.jsp |  213 ++
 .../WEB-INF/partials/canvas/registration.jsp    |   44 +
 .../canvas/remote-port-configuration.jsp        |   42 +
 .../remote-process-group-configuration.jsp      |   50 +
 .../canvas/remote-process-group-details.jsp     |   48 +
 .../canvas/remote-process-group-ports.jsp       |   47 +
 .../canvas/secure-port-configuration.jsp        |   82 +
 .../partials/canvas/secure-port-details.jsp     |   67 +
 .../partials/canvas/settings-content.jsp        |   60 +
 .../webapp/WEB-INF/partials/canvas/shell.jsp    |   27 +
 .../partials/cluster/cluster-content.jsp        |   39 +
 .../partials/cluster/node-details-dialog.jsp    |   42 +
 .../WEB-INF/partials/connection-details.jsp     |  123 +
 .../partials/counters/counters-content.jsp      |   39 +
 .../partials/history/action-details-dialog.jsp  |   27 +
 .../partials/history/history-content.jsp        |   36 +
 .../partials/history/history-filter-dialog.jsp  |   73 +
 .../partials/history/history-purge-dialog.jsp   |   42 +
 .../webapp/WEB-INF/partials/message-pane.jsp    |   23 +
 .../main/webapp/WEB-INF/partials/ok-dialog.jsp  |   22 +
 .../WEB-INF/partials/processor-details.jsp      |  158 +
 .../provenance/lineage-query-dialog.jsp         |   26 +
 .../partials/provenance/provenance-content.jsp  |   73 +
 .../provenance-event-details-dialog.jsp         |  196 +
 .../provenance/provenance-query-dialog.jsp      |   26 +
 .../provenance/provenance-search-dialog.jsp     |  101 +
 .../WEB-INF/partials/status-history-dialog.jsp  |   37 +
 .../cluster-connection-summary-dialog.jsp       |   36 +
 .../cluster-input-port-summary-dialog.jsp       |   36 +
 .../cluster-output-port-summary-dialog.jsp      |   36 +
 .../cluster-processor-summary-dialog.jsp        |   36 +
 ...ster-remote-process-group-summary-dialog.jsp |   36 +
 .../partials/summary/summary-content.jsp        |   64 +
 .../summary/system-diagnostics-dialog.jsp       |  131 +
 .../summary/view-single-node-dialog.jsp         |   24 +
 .../partials/templates/templates-content.jsp    |   59 +
 .../partials/users/group-revoke-dialog.jsp      |   22 +
 .../partials/users/group-roles-dialog.jsp       |   52 +
 .../partials/users/user-delete-dialog.jsp       |   23 +
 .../partials/users/user-details-dialog.jsp      |   56 +
 .../partials/users/user-group-dialog.jsp        |   27 +
 .../partials/users/user-revoke-dialog.jsp       |   23 +
 .../partials/users/user-roles-dialog.jsp        |   60 +
 .../WEB-INF/partials/users/users-content.jsp    |   46 +
 .../webapp/WEB-INF/partials/yes-no-dialog.jsp   |   22 +
 .../nifi-web-ui/src/main/webapp/WEB-INF/web.xml |  120 +
 .../nifi-web-ui/src/main/webapp/css/about.css   |   54 +
 .../nifi-web-ui/src/main/webapp/css/banner.css  |   80 +
 .../src/main/webapp/css/bulletin-board.css      |  238 ++
 .../nifi-web-ui/src/main/webapp/css/canvas.css  |   41 +
 .../nifi-web-ui/src/main/webapp/css/cluster.css |  152 +
 .../webapp/css/connection-configuration.css     |  168 +
 .../src/main/webapp/css/connection-details.css  |   77 +
 .../src/main/webapp/css/counters.css            |  134 +
 .../nifi-web-ui/src/main/webapp/css/dialog.css  |  351 ++
 .../src/main/webapp/css/flow-status.css         |  110 +
 .../nifi-web-ui/src/main/webapp/css/graph.css   |  262 ++
 .../nifi-web-ui/src/main/webapp/css/header.css  |  683 ++++
 .../nifi-web-ui/src/main/webapp/css/history.css |  216 ++
 .../src/main/webapp/css/label-configuration.css |   55 +
 .../nifi-web-ui/src/main/webapp/css/main.css    |  298 ++
 .../src/main/webapp/css/message-page.css        |   30 +
 .../src/main/webapp/css/message-pane.css        |   44 +
 .../src/main/webapp/css/navigation.css          |  292 ++
 .../main/webapp/css/new-processor-dialog.css    |  168 +
 .../src/main/webapp/css/port-configuration.css  |  192 +
 .../src/main/webapp/css/port-details.css        |   55 +
 .../webapp/css/process-group-configuration.css  |   40 +
 .../main/webapp/css/process-group-details.css   |   27 +
 .../main/webapp/css/processor-configuration.css |  388 ++
 .../src/main/webapp/css/processor-details.css   |  119 +
 .../src/main/webapp/css/provenance.css          |  654 ++++
 .../src/main/webapp/css/registration.css        |   53 +
 .../css/remote-process-group-configuration.css  |  233 ++
 .../nifi-web-ui/src/main/webapp/css/reset.css   |   43 +
 .../src/main/webapp/css/settings.css            |   66 +
 .../nifi-web-ui/src/main/webapp/css/shell.css   |   98 +
 .../src/main/webapp/css/status-history.css      |  199 +
 .../nifi-web-ui/src/main/webapp/css/summary.css |  768 ++++
 .../src/main/webapp/css/templates.css           |  229 ++
 .../nifi-web-ui/src/main/webapp/css/users.css   |  254 ++
 .../src/main/webapp/images/addConnect.png       |  Bin 0 -> 1996 bytes
 .../src/main/webapp/images/bg-error.png         |  Bin 0 -> 63766 bytes
 .../src/main/webapp/images/bgBannerFoot.png     |  Bin 0 -> 189 bytes
 .../src/main/webapp/images/bgBannerHead.png     |  Bin 0 -> 422 bytes
 .../src/main/webapp/images/bgBreadcrumbs.png    |  Bin 0 -> 2130 bytes
 .../src/main/webapp/images/bgButton.png         |  Bin 0 -> 234 bytes
 .../src/main/webapp/images/bgButtonOver.png     |  Bin 0 -> 234 bytes
 .../src/main/webapp/images/bgButtonSelected.png |  Bin 0 -> 166 bytes
 .../main/webapp/images/bgButtonSelectedOver.png |  Bin 0 -> 168 bytes
 .../src/main/webapp/images/bgContextMenu.png    |  Bin 0 -> 746 bytes
 .../src/main/webapp/images/bgControlsInset.png  |  Bin 0 -> 618 bytes
 .../src/main/webapp/images/bgHeader.png         |  Bin 0 -> 1697 bytes
 .../src/main/webapp/images/bgInputText.png      |  Bin 0 -> 139 bytes
 .../src/main/webapp/images/bgNifiLogo.png       |  Bin 0 -> 4232 bytes
 .../src/main/webapp/images/bgNifiToolbar.png    |  Bin 0 -> 2123 bytes
 .../main/webapp/images/bgNifiToolbarEdge.png    |  Bin 0 -> 541 bytes
 .../src/main/webapp/images/bgOutline.png        |  Bin 0 -> 585 bytes
 .../src/main/webapp/images/bgPanCenter.png      |  Bin 0 -> 362 bytes
 .../src/main/webapp/images/bgPanZoom.png        |  Bin 0 -> 832 bytes
 .../webapp/images/bgProcessGroupDetailsArea.png |  Bin 0 -> 6596 bytes
 .../main/webapp/images/bgProcessorStatArea.png  |  Bin 0 -> 4062 bytes
 .../images/bgRemoteProcessGroupDetailsArea.png  |  Bin 0 -> 4433 bytes
 .../src/main/webapp/images/bgSearchInput.png    |  Bin 0 -> 221 bytes
 .../src/main/webapp/images/bgShellClose.png     |  Bin 0 -> 169 bytes
 .../src/main/webapp/images/bgStatusPanel.png    |  Bin 0 -> 229 bytes
 .../src/main/webapp/images/bgTabContainer.png   |  Bin 0 -> 234 bytes
 .../src/main/webapp/images/bgTableHeader.png    |  Bin 0 -> 232 bytes
 .../main/webapp/images/bgToolbarBtnBorder.png   |  Bin 0 -> 227 bytes
 .../src/main/webapp/images/blueBtnBg-over.jpg   |  Bin 0 -> 356 bytes
 .../src/main/webapp/images/blueBtnBg.jpg        |  Bin 0 -> 356 bytes
 .../src/main/webapp/images/buttonColor.png      |  Bin 0 -> 1679 bytes
 .../src/main/webapp/images/buttonCopy.png       |  Bin 0 -> 1166 bytes
 .../src/main/webapp/images/buttonDelete.png     |  Bin 0 -> 1837 bytes
 .../src/main/webapp/images/buttonDisable.png    |  Bin 0 -> 1737 bytes
 .../src/main/webapp/images/buttonEnable.png     |  Bin 0 -> 1375 bytes
 .../src/main/webapp/images/buttonGroup.png      |  Bin 0 -> 880 bytes
 .../main/webapp/images/buttonNewProperty.png    |  Bin 0 -> 590 bytes
 .../src/main/webapp/images/buttonOutline.png    |  Bin 0 -> 495 bytes
 .../src/main/webapp/images/buttonPanDown.png    |  Bin 0 -> 554 bytes
 .../src/main/webapp/images/buttonPanLeft.png    |  Bin 0 -> 588 bytes
 .../src/main/webapp/images/buttonPanRight.png   |  Bin 0 -> 530 bytes
 .../src/main/webapp/images/buttonPanUp.png      |  Bin 0 -> 551 bytes
 .../src/main/webapp/images/buttonPaste.png      |  Bin 0 -> 1372 bytes
 .../src/main/webapp/images/buttonRefresh.png    |  Bin 0 -> 915 bytes
 .../src/main/webapp/images/buttonRun.png        |  Bin 0 -> 1163 bytes
 .../src/main/webapp/images/buttonStop.png       |  Bin 0 -> 869 bytes
 .../src/main/webapp/images/buttonTemplate.png   |  Bin 0 -> 1290 bytes
 .../src/main/webapp/images/buttonZoom100.png    |  Bin 0 -> 449 bytes
 .../src/main/webapp/images/buttonZoomFit.png    |  Bin 0 -> 499 bytes
 .../src/main/webapp/images/buttonZoomIn.png     |  Bin 0 -> 435 bytes
 .../src/main/webapp/images/buttonZoomOut.png    |  Bin 0 -> 339 bytes
 .../src/main/webapp/images/cluster.png          |  Bin 0 -> 326 bytes
 .../src/main/webapp/images/grayBtnBg-over.jpg   |  Bin 0 -> 319 bytes
 .../src/main/webapp/images/grayBtnBg.jpg        |  Bin 0 -> 317 bytes
 .../nifi-web-ui/src/main/webapp/images/grid.gif |  Bin 0 -> 135 bytes
 .../src/main/webapp/images/headerTabBg.gif      |  Bin 0 -> 131 bytes
 .../src/main/webapp/images/iconAdminUser.png    |  Bin 0 -> 1960 bytes
 .../src/main/webapp/images/iconAlert.png        |  Bin 0 -> 1396 bytes
 .../src/main/webapp/images/iconAlertDialog.png  |  Bin 0 -> 1241 bytes
 .../src/main/webapp/images/iconAutoRefresh.png  |  Bin 0 -> 3102 bytes
 .../src/main/webapp/images/iconBulletin.png     |  Bin 0 -> 1066 bytes
 .../main/webapp/images/iconBulletinBoard.png    |  Bin 0 -> 2523 bytes
 .../src/main/webapp/images/iconCenterView.png   |  Bin 0 -> 338 bytes
 .../src/main/webapp/images/iconChart.png        |  Bin 0 -> 510 bytes
 .../src/main/webapp/images/iconClose.png        |  Bin 0 -> 231 bytes
 .../src/main/webapp/images/iconCluster.png      |  Bin 0 -> 2112 bytes
 .../src/main/webapp/images/iconClusterSmall.png |  Bin 0 -> 757 bytes
 .../main/webapp/images/iconCollapseChildren.png |  Bin 0 -> 571 bytes
 .../main/webapp/images/iconCollapseParents.png  |  Bin 0 -> 645 bytes
 .../src/main/webapp/images/iconColor.png        |  Bin 0 -> 738 bytes
 .../src/main/webapp/images/iconCommit.png       |  Bin 0 -> 445 bytes
 .../src/main/webapp/images/iconConfigure.png    |  Bin 0 -> 696 bytes
 .../src/main/webapp/images/iconConnect.png      |  Bin 0 -> 589 bytes
 .../src/main/webapp/images/iconConnection.png   |  Bin 0 -> 1517 bytes
 .../src/main/webapp/images/iconCopy.png         |  Bin 0 -> 514 bytes
 .../src/main/webapp/images/iconCounter.png      |  Bin 0 -> 1369 bytes
 .../src/main/webapp/images/iconDelete.png       |  Bin 0 -> 670 bytes
 .../src/main/webapp/images/iconDetails.png      |  Bin 0 -> 549 bytes
 .../src/main/webapp/images/iconDisable.png      |  Bin 0 -> 764 bytes
 .../src/main/webapp/images/iconDisconnect.png   |  Bin 0 -> 569 bytes
 .../src/main/webapp/images/iconEdit.png         |  Bin 0 -> 493 bytes
 .../src/main/webapp/images/iconEditButton.png   |  Bin 0 -> 915 bytes
 .../main/webapp/images/iconExpandChildren.png   |  Bin 0 -> 521 bytes
 .../main/webapp/images/iconExpandParents.png    |  Bin 0 -> 585 bytes
 .../src/main/webapp/images/iconExport.png       |  Bin 0 -> 453 bytes
 .../main/webapp/images/iconExportLineage.png    |  Bin 0 -> 405 bytes
 .../src/main/webapp/images/iconFlowSettings.png |  Bin 0 -> 2624 bytes
 .../src/main/webapp/images/iconFunnel.png       |  Bin 0 -> 1223 bytes
 .../src/main/webapp/images/iconGoTo.png         |  Bin 0 -> 448 bytes
 .../src/main/webapp/images/iconHistory.png      |  Bin 0 -> 2347 bytes
 .../src/main/webapp/images/iconInfo.png         |  Bin 0 -> 550 bytes
 .../src/main/webapp/images/iconInputPort.png    |  Bin 0 -> 1842 bytes
 .../main/webapp/images/iconInputPortSmall.png   |  Bin 0 -> 532 bytes
 .../webapp/images/iconIsolatedProcessor.png     |  Bin 0 -> 1781 bytes
 .../src/main/webapp/images/iconLineage.png      |  Bin 0 -> 432 bytes
 .../src/main/webapp/images/iconLoading.gif      |  Bin 0 -> 2438 bytes
 .../src/main/webapp/images/iconNotSecure.png    |  Bin 0 -> 137 bytes
 .../src/main/webapp/images/iconOutputPort.png   |  Bin 0 -> 1658 bytes
 .../main/webapp/images/iconOutputPortSmall.png  |  Bin 0 -> 459 bytes
 .../src/main/webapp/images/iconPaste.png        |  Bin 0 -> 601 bytes
 .../src/main/webapp/images/iconPopOut.png       |  Bin 0 -> 217 bytes
 .../webapp/images/iconPortNotTransmitting.png   |  Bin 0 -> 231 bytes
 .../src/main/webapp/images/iconPortRemoved.png  |  Bin 0 -> 456 bytes
 .../main/webapp/images/iconPortTransmitting.png |  Bin 0 -> 235 bytes
 .../src/main/webapp/images/iconPrimary.png      |  Bin 0 -> 647 bytes
 .../src/main/webapp/images/iconProcessor.png    |  Bin 0 -> 1446 bytes
 .../src/main/webapp/images/iconProvenance.png   |  Bin 0 -> 1104 bytes
 .../src/main/webapp/images/iconRefresh.png      |  Bin 0 -> 492 bytes
 .../src/main/webapp/images/iconRemotePorts.png  |  Bin 0 -> 456 bytes
 .../webapp/images/iconRemoteProcessGroup.png    |  Bin 0 -> 674 bytes
 .../src/main/webapp/images/iconReporting.png    |  Bin 0 -> 803 bytes
 .../src/main/webapp/images/iconResetCounter.png |  Bin 0 -> 304 bytes
 .../src/main/webapp/images/iconResize.png       |  Bin 0 -> 142 bytes
 .../src/main/webapp/images/iconRevoke.png       |  Bin 0 -> 676 bytes
 .../src/main/webapp/images/iconRun.png          |  Bin 0 -> 538 bytes
 .../src/main/webapp/images/iconSecure.png       |  Bin 0 -> 133 bytes
 .../src/main/webapp/images/iconSmallFunnel.png  |  Bin 0 -> 455 bytes
 .../main/webapp/images/iconSmallInputPort.png   |  Bin 0 -> 216 bytes
 .../main/webapp/images/iconSmallOutputPort.png  |  Bin 0 -> 264 bytes
 .../webapp/images/iconSmallProcessGroup.png     |  Bin 0 -> 479 bytes
 .../main/webapp/images/iconSmallProcessor.png   |  Bin 0 -> 647 bytes
 .../webapp/images/iconSmallRelationship.png     |  Bin 0 -> 770 bytes
 .../images/iconSmallRemoteProcessGroup.png      |  Bin 0 -> 402 bytes
 .../src/main/webapp/images/iconStop.png         |  Bin 0 -> 402 bytes
 .../src/main/webapp/images/iconTemplates.png    |  Bin 0 -> 2659 bytes
 .../src/main/webapp/images/iconToFront.png      |  Bin 0 -> 475 bytes
 .../webapp/images/iconTransmissionActive.png    |  Bin 0 -> 1330 bytes
 .../webapp/images/iconTransmissionInactive.png  |  Bin 0 -> 1248 bytes
 .../src/main/webapp/images/iconTwistArrow.png   |  Bin 0 -> 256 bytes
 .../src/main/webapp/images/iconUndo.png         |  Bin 0 -> 642 bytes
 .../src/main/webapp/images/iconUsage.png        |  Bin 0 -> 470 bytes
 .../src/main/webapp/images/inputCheckbox.png    |  Bin 0 -> 389 bytes
 .../src/main/webapp/images/loadAnimation.gif    |  Bin 0 -> 10789 bytes
 .../src/main/webapp/images/nifi16.ico           |  Bin 0 -> 1150 bytes
 .../src/main/webapp/images/nifi_about.png       |  Bin 0 -> 16323 bytes
 .../src/main/webapp/images/panelBg.jpg          |  Bin 0 -> 312 bytes
 .../src/main/webapp/images/portRemoved.png      |  Bin 0 -> 1090 bytes
 .../src/main/webapp/images/portRunning.png      |  Bin 0 -> 337 bytes
 .../src/main/webapp/images/portStopped.png      |  Bin 0 -> 192 bytes
 .../src/main/webapp/images/spacer.png           |  Bin 0 -> 110 bytes
 .../src/main/webapp/images/starburst.png        |  Bin 0 -> 190 bytes
 .../src/main/webapp/images/tabBg.jpg            |  Bin 0 -> 320 bytes
 .../src/main/webapp/images/toolbarBg.gif        |  Bin 0 -> 45 bytes
 .../main/webapp/images/toolboxIconFunnel.png    |  Bin 0 -> 2409 bytes
 .../main/webapp/images/toolboxIconInputPort.png |  Bin 0 -> 2680 bytes
 .../src/main/webapp/images/toolboxIconLabel.png |  Bin 0 -> 2871 bytes
 .../webapp/images/toolboxIconOutputPort.png     |  Bin 0 -> 2209 bytes
 .../webapp/images/toolboxIconProcessGroup.png   |  Bin 0 -> 4176 bytes
 .../main/webapp/images/toolboxIconProcessor.png |  Bin 0 -> 4426 bytes
 .../images/toolboxIconRemoteProcessGroup.png    |  Bin 0 -> 5653 bytes
 .../main/webapp/images/toolboxIconTemplate.png  |  Bin 0 -> 2913 bytes
 .../images/transmissionSwitchDisabled.png       |  Bin 0 -> 1240 bytes
 .../webapp/images/transmissionSwitchEnabled.png |  Bin 0 -> 1600 bytes
 .../src/main/webapp/images/ungroup.png          |  Bin 0 -> 3409 bytes
 .../src/main/webapp/js/codemirror/LICENSE       |   19 +
 .../js/codemirror/addon/hint/show-hint.css      |   38 +
 .../js/codemirror/lib/codemirror-compressed.js  |   16 +
 .../webapp/js/codemirror/lib/codemirror.css     |  318 ++
 .../nifi-web-ui/src/main/webapp/js/d3/LICENSE   |   26 +
 .../nifi-web-ui/src/main/webapp/js/d3/d3.min.js |    5 +
 .../webapp/js/jquery/combo/jquery.combo.css     |   88 +
 .../main/webapp/js/jquery/combo/jquery.combo.js |  311 ++
 .../main/webapp/js/jquery/combo/menuArrow.png   |  Bin 0 -> 251 bytes
 .../webapp/js/jquery/combo/menuArrowSmall.png   |  Bin 0 -> 233 bytes
 .../main/webapp/js/jquery/jquery-2.1.1.min.js   |    4 +
 .../src/main/webapp/js/jquery/jquery.center.js  |   35 +
 .../src/main/webapp/js/jquery/jquery.count.js   |   99 +
 .../src/main/webapp/js/jquery/jquery.each.js    |   31 +
 .../main/webapp/js/jquery/jquery.ellipsis.js    |  164 +
 .../js/jquery/jquery.event.drag-2.2.min.js      |    6 +
 .../main/webapp/js/jquery/jquery.form.min.js    |   11 +
 .../src/main/webapp/js/jquery/jquery.tab.js     |   69 +
 .../js/jquery/minicolors/jquery.minicolors.css  |  259 ++
 .../jquery/minicolors/jquery.minicolors.min.js  |    9 +
 .../js/jquery/minicolors/jquery.minicolors.png  |  Bin 0 -> 77459 bytes
 .../webapp/js/jquery/modal/jquery.modal.css     |  110 +
 .../main/webapp/js/jquery/modal/jquery.modal.js |  266 ++
 .../js/jquery/nfeditor/jquery.nfeditor.css      |   73 +
 .../js/jquery/nfeditor/jquery.nfeditor.js       |  315 ++
 .../js/jquery/nfeditor/languages/nfel.css       |   51 +
 .../webapp/js/jquery/nfeditor/languages/nfel.js |  831 ++++
 .../webapp/js/jquery/qtip2/jquery.qtip.min.css  |    3 +
 .../webapp/js/jquery/qtip2/jquery.qtip.min.js   |    5 +
 .../webapp/js/jquery/slickgrid/MIT-LICENSE.txt  |   20 +
 .../js/jquery/slickgrid/css/images/collapse.gif |  Bin 0 -> 846 bytes
 .../js/jquery/slickgrid/css/images/expand.gif   |  Bin 0 -> 851 bytes
 .../js/jquery/slickgrid/css/images/sort-asc.gif |  Bin 0 -> 59 bytes
 .../jquery/slickgrid/css/images/sort-desc.gif   |  Bin 0 -> 59 bytes
 .../slickgrid/css/slick-default-theme.css       |  135 +
 .../js/jquery/slickgrid/css/slick.grid.css      |  158 +
 .../slickgrid/plugins/slick.autotooltips.js     |   83 +
 .../plugins/slick.cellrangedecorator.js         |   66 +
 .../plugins/slick.cellrangeselector.js          |  113 +
 .../plugins/slick.cellselectionmodel.js         |  154 +
 .../plugins/slick.rowselectionmodel.js          |  187 +
 .../webapp/js/jquery/slickgrid/slick.core.js    |  467 +++
 .../js/jquery/slickgrid/slick.dataview.js       | 1126 ++++++
 .../webapp/js/jquery/slickgrid/slick.editors.js |  512 +++
 .../js/jquery/slickgrid/slick.formatters.js     |   59 +
 .../webapp/js/jquery/slickgrid/slick.grid.js    | 3422 +++++++++++++++++
 .../webapp/js/jquery/tabbs/jquery.tabbs.css     |   48 +
 .../main/webapp/js/jquery/tabbs/jquery.tabbs.js |   96 +
 .../ui-smoothness/images/animated-overlay.gif   |  Bin 0 -> 1738 bytes
 .../images/ui-bg_flat_0_aaaaaa_40x100.png       |  Bin 0 -> 212 bytes
 .../images/ui-bg_flat_75_ffffff_40x100.png      |  Bin 0 -> 208 bytes
 .../images/ui-bg_glass_55_fbf9ee_1x400.png      |  Bin 0 -> 335 bytes
 .../images/ui-bg_glass_65_ffffff_1x400.png      |  Bin 0 -> 207 bytes
 .../images/ui-bg_glass_75_dadada_1x400.png      |  Bin 0 -> 262 bytes
 .../images/ui-bg_glass_75_e6e6e6_1x400.png      |  Bin 0 -> 262 bytes
 .../images/ui-bg_glass_95_fef1ec_1x400.png      |  Bin 0 -> 332 bytes
 .../ui-bg_highlight-soft_75_cccccc_1x100.png    |  Bin 0 -> 280 bytes
 .../images/ui-icons_222222_256x240.png          |  Bin 0 -> 6922 bytes
 .../images/ui-icons_2e83ff_256x240.png          |  Bin 0 -> 4549 bytes
 .../images/ui-icons_454545_256x240.png          |  Bin 0 -> 6992 bytes
 .../images/ui-icons_888888_256x240.png          |  Bin 0 -> 6999 bytes
 .../images/ui-icons_cd0a0a_256x240.png          |  Bin 0 -> 4549 bytes
 .../ui-smoothness/jquery-ui-1.10.4.min.css      |    7 +
 .../ui-smoothness/jquery-ui-1.10.4.min.js       |    7 +
 .../web/nifi-web-ui/src/main/webapp/js/json2.js |  475 +++
 .../js/nf/bulletin-board/nf-bulletin-board.js   |  422 ++
 .../src/main/webapp/js/nf/canvas/nf-actions.js  | 1150 ++++++
 .../src/main/webapp/js/nf/canvas/nf-birdseye.js |  340 ++
 .../webapp/js/nf/canvas/nf-canvas-header.js     |  293 ++
 .../webapp/js/nf/canvas/nf-canvas-toolbar.js    |  174 +
 .../webapp/js/nf/canvas/nf-canvas-toolbox.js    | 1211 ++++++
 .../main/webapp/js/nf/canvas/nf-canvas-utils.js | 1307 +++++++
 .../src/main/webapp/js/nf/canvas/nf-canvas.js   | 1530 ++++++++
 .../main/webapp/js/nf/canvas/nf-clipboard.js    |   96 +
 .../main/webapp/js/nf/canvas/nf-connectable.js  |  210 +
 .../js/nf/canvas/nf-connection-configuration.js | 1381 +++++++
 .../main/webapp/js/nf/canvas/nf-connection.js   | 1533 ++++++++
 .../main/webapp/js/nf/canvas/nf-context-menu.js |  452 +++
 .../js/nf/canvas/nf-custom-processor-ui.js      |   43 +
 .../main/webapp/js/nf/canvas/nf-draggable.js    |  319 ++
 .../src/main/webapp/js/nf/canvas/nf-funnel.js   |  279 ++
 .../src/main/webapp/js/nf/canvas/nf-go-to.js    |  730 ++++
 .../webapp/js/nf/canvas/nf-graph-control.js     |  137 +
 .../src/main/webapp/js/nf/canvas/nf-graph.js    |  194 +
 .../js/nf/canvas/nf-label-configuration.js      |  132 +
 .../src/main/webapp/js/nf/canvas/nf-label.js    |  559 +++
 .../js/nf/canvas/nf-port-configuration.js       |  170 +
 .../main/webapp/js/nf/canvas/nf-port-details.js |   63 +
 .../src/main/webapp/js/nf/canvas/nf-port.js     |  632 +++
 .../nf/canvas/nf-process-group-configuration.js |  107 +
 .../js/nf/canvas/nf-process-group-details.js    |   61 +
 .../webapp/js/nf/canvas/nf-process-group.js     | 1064 +++++
 .../js/nf/canvas/nf-processor-configuration.js  |  734 ++++
 .../nf-processor-property-combo-editor.js       |  175 +
 .../canvas/nf-processor-property-nfel-editor.js |  207 +
 .../js/nf/canvas/nf-processor-property-table.js |  567 +++
 .../canvas/nf-processor-property-text-editor.js |  212 +
 .../main/webapp/js/nf/canvas/nf-processor.js    |  831 ++++
 .../main/webapp/js/nf/canvas/nf-registration.js |   68 +
 .../nf-remote-process-group-configuration.js    |  124 +
 .../canvas/nf-remote-process-group-details.js   |   67 +
 .../nf/canvas/nf-remote-process-group-ports.js  |  525 +++
 .../js/nf/canvas/nf-remote-process-group.js     | 1060 +++++
 .../src/main/webapp/js/nf/canvas/nf-search.js   |  179 +
 .../nf/canvas/nf-secure-port-configuration.js   |  384 ++
 .../js/nf/canvas/nf-secure-port-details.js      |  118 +
 .../main/webapp/js/nf/canvas/nf-selectable.js   |   58 +
 .../src/main/webapp/js/nf/canvas/nf-settings.js |  146 +
 .../src/main/webapp/js/nf/canvas/nf-snippet.js  |  212 +
 .../src/main/webapp/js/nf/canvas/nf-storage.js  |  136 +
 .../webapp/js/nf/canvas/nf-toolbar-action.js    |   86 +
 .../webapp/js/nf/cluster/nf-cluster-table.js    |  620 +++
 .../src/main/webapp/js/nf/cluster/nf-cluster.js |  154 +
 .../webapp/js/nf/counters/nf-counters-table.js  |  294 ++
 .../main/webapp/js/nf/counters/nf-counters.js   |  153 +
 .../webapp/js/nf/history/nf-history-model.js    |  218 ++
 .../webapp/js/nf/history/nf-history-table.js    |  429 +++
 .../src/main/webapp/js/nf/history/nf-history.js |  154 +
 .../src/main/webapp/js/nf/nf-client.js          |   56 +
 .../src/main/webapp/js/nf/nf-common.js          |  899 +++++
 .../main/webapp/js/nf/nf-connection-details.js  |  467 +++
 .../src/main/webapp/js/nf/nf-dialog.js          |  128 +
 .../src/main/webapp/js/nf/nf-namespace.js       |   20 +
 .../main/webapp/js/nf/nf-processor-details.js   |  586 +++
 .../src/main/webapp/js/nf/nf-shell.js           |  167 +
 .../src/main/webapp/js/nf/nf-status-history.js  | 1320 +++++++
 .../js/nf/provenance/nf-provenance-lineage.js   | 1413 +++++++
 .../js/nf/provenance/nf-provenance-table.js     | 1371 +++++++
 .../webapp/js/nf/provenance/nf-provenance.js    |  204 +
 .../webapp/js/nf/summary/nf-cluster-search.js   |  183 +
 .../webapp/js/nf/summary/nf-summary-table.js    | 2436 ++++++++++++
 .../src/main/webapp/js/nf/summary/nf-summary.js |  174 +
 .../js/nf/templates/nf-templates-table.js       |  332 ++
 .../main/webapp/js/nf/templates/nf-templates.js |  235 ++
 .../main/webapp/js/nf/users/nf-users-table.js   | 1090 ++++++
 .../src/main/webapp/js/nf/users/nf-users.js     |  149 +
 .../framework-bundle/framework/web/pom.xml      |   74 +
 .../web/web-optimistic-locking/.gitignore       |    1 +
 .../web/web-optimistic-locking/pom.xml          |   32 +
 .../apache/nifi/web/ConfigurationSnapshot.java  |   66 +
 .../nifi/web/OptimisticLockingManager.java      |   95 +
 .../web/StandardOptimisticLockingManager.java   |   77 +
 .../framework/web/web-security/.gitignore       |    1 +
 .../framework/web/web-security/pom.xml          |   81 +
 .../org/apache/nifi/web/security/DnUtils.java   |   87 +
 .../web/security/UntrustedProxyException.java   |   34 +
 .../anonymous/NiFiAnonymousUserFilter.java      |  101 +
 .../NiFiAuthenticationEntryPoint.java           |   68 +
 .../authorization/NiFiAuthorizationService.java |  173 +
 .../authorization/NodeAuthorizedUserFilter.java |  128 +
 .../nifi/web/security/user/NiFiUserDetails.java |  110 +
 .../nifi/web/security/user/NiFiUserUtils.java   |   82 +
 .../x509/SubjectDnX509PrincipalExtractor.java   |   38 +
 .../security/x509/X509AuthenticationFilter.java |  349 ++
 .../security/x509/X509CertificateExtractor.java |   52 +
 .../x509/ocsp/CertificateStatusException.java   |   42 +
 .../x509/ocsp/OcspCertificateValidator.java     |  444 +++
 .../web/security/x509/ocsp/OcspRequest.java     |   69 +
 .../nifi/web/security/x509/ocsp/OcspStatus.java |   82 +
 .../resources/nifi-web-security-context.xml     |   83 +
 .../NiFiAuthorizationServiceTest.java           |  250 ++
 .../nar-bundles/framework-bundle/nar/.gitignore |    1 +
 nifi/nar-bundles/framework-bundle/nar/pom.xml   |   66 +
 nifi/nar-bundles/framework-bundle/pom.xml       |  114 +
 .../hadoop-bundle/hdfs-processors/pom.xml       |   65 +
 .../hadoop/AbstractHadoopProcessor.java         |  221 ++
 .../hadoop/CreateHadoopSequenceFile.java        |  177 +
 ...lowFileStreamUnpackerSequenceFileWriter.java |  170 +
 .../apache/nifi/processors/hadoop/GetHDFS.java  |  563 +++
 .../processors/hadoop/GetHDFSSequenceFile.java  |  146 +
 .../nifi/processors/hadoop/KeyValueReader.java  |  132 +
 .../apache/nifi/processors/hadoop/PutHDFS.java  |  403 ++
 .../hadoop/SequenceFileWriterImpl.java          |  119 +
 .../hadoop/TarUnpackerSequenceFileWriter.java   |   54 +
 .../nifi/processors/hadoop/ValueReader.java     |  116 +
 .../hadoop/ZipUnpackerSequenceFileWriter.java   |   57 +
 .../hadoop/util/ByteFilteringOutputStream.java  |  163 +
 .../hadoop/util/InputStreamWritable.java        |   61 +
 .../hadoop/util/OutputStreamWritable.java       |   85 +
 .../hadoop/util/SequenceFileReader.java         |   28 +
 .../hadoop/util/SequenceFileWriter.java         |   40 +
 .../org.apache.nifi.processor.Processor         |   18 +
 .../index.html                                  |   88 +
 .../index.html                                  |  162 +
 .../index.html                                  |  150 +
 .../index.html                                  |  159 +
 .../processors/hadoop/AbstractHadoopTest.java   |   90 +
 .../nifi/processors/hadoop/GetHDFSTest.java     |  123 +
 .../nifi/processors/hadoop/PutHDFSTest.java     |  158 +
 .../hadoop/SimpleHadoopProcessor.java           |   30 +
 .../hadoop/TestCreateHadoopSequenceFile.java    |  176 +
 .../src/test/resources/core-site-broken.xml     |   25 +
 .../src/test/resources/core-site.xml            |   25 +
 .../resources/testdata/13545312236534130.tar    |  Bin 0 -> 1505280 bytes
 .../resources/testdata/13545423550275052.zip    |  Bin 0 -> 1500841 bytes
 .../resources/testdata/13545479542069498.pkg    |  Bin 0 -> 1500429 bytes
 .../src/test/resources/testdata/randombytes-1   |  Bin 0 -> 500000 bytes
 .../src/test/resources/testdata/randombytes-2   |  Bin 0 -> 500000 bytes
 .../src/test/resources/testdata/randombytes-3   |  Bin 0 -> 500000 bytes
 nifi/nar-bundles/hadoop-bundle/nar/pom.xml      |   41 +
 nifi/nar-bundles/hadoop-bundle/pom.xml          |   46 +
 .../hadoop-libraries-bundle/nar/pom.xml         |   45 +
 .../nar-bundles/hadoop-libraries-bundle/pom.xml |   36 +
 nifi/nar-bundles/jetty-bundle/pom.xml           |   82 +
 nifi/nar-bundles/kafka-bundle/kafka-nar/pom.xml |   37 +
 .../kafka-bundle/kafka-processors/pom.xml       |   76 +
 .../apache/nifi/processors/kafka/GetKafka.java  |  330 ++
 .../apache/nifi/processors/kafka/PutKafka.java  |  419 ++
 .../org.apache.nifi.processor.Processor         |   16 +
 .../index.html                                  |  173 +
 .../index.html                                  |  189 +
 .../nifi/processors/kafka/TestGetKafka.java     |  162 +
 .../nifi/processors/kafka/TestPutKafka.java     |  236 ++
 nifi/nar-bundles/kafka-bundle/pom.xml           |   35 +
 .../monitor-threshold-bundle/nar/pom.xml        |   39 +
 .../monitor-threshold-bundle/pom.xml            |   51 +
 .../monitor-threshold-bundle/processor/pom.xml  |   45 +
 .../processors/monitor/MonitorThreshold.java    |  904 +++++
 .../processors/monitor/ThresholdsParser.java    |  134 +
 .../org.apache.nifi.processor.Processor         |   15 +
 .../index.html                                  |  485 +++
 .../monitor/TestMonitorThreshold.java           | 1560 ++++++++
 .../processor/src/test/resources/testFile       |   23 +
 .../resources/threshold_settings_allow_0.xml    |   21 +
 .../resources/threshold_settings_allow_1.xml    |   21 +
 .../threshold_settings_allow_10_KB.xml          |   21 +
 .../threshold_settings_allow_10_KB_and_2_KB.xml |   26 +
 .../threshold_settings_allow_1669_bytes.xml     |   21 +
 .../resources/threshold_settings_allow_2.xml    |   21 +
 .../threshold_settings_allow_2_default.xml      |   21 +
 ...eshold_settings_allow_3336_bytes_default.xml |   21 +
 ...hold_settings_with_empty_attribute_value.xml |   21 +
 ...ettings_with_spaces_for_attribute_values.xml |   22 +
 .../ui/nb-configuration.xml                     |   18 +
 .../monitor-threshold-bundle/ui/pom.xml         |  134 +
 .../thresholds/ui/AttributeComparator.java      |   36 +
 .../thresholds/ui/AttributeResource.java        |  242 ++
 .../monitor/thresholds/ui/RuleComparator.java   |   46 +
 .../thresholds/ui/ThresholdResource.java        |  227 ++
 .../ui/ThresholdSettingsResource.java           |  191 +
 .../thresholds/ui/ThresholdsConfigFile.java     |  296 ++
 .../ui/src/main/webapp/META-INF/nifi-processor  |   15 +
 .../webapp/WEB-INF/jsp/attribute-add-dialog.jsp |   43 +
 .../WEB-INF/jsp/attribute-confirm-dialog.jsp    |   21 +
 .../WEB-INF/jsp/attribute-edit-dialog.jsp       |   27 +
 .../WEB-INF/jsp/attribute-filter-dialog.jsp     |   26 +
 .../main/webapp/WEB-INF/jsp/error-dialog.jsp    |   21 +
 .../ui/src/main/webapp/WEB-INF/jsp/index.jsp    |  101 +
 .../webapp/WEB-INF/jsp/threshold-add-dialog.jsp |   45 +
 .../WEB-INF/jsp/threshold-confirm-dialog.jsp    |   21 +
 .../webapp/WEB-INF/jsp/threshold-dialog.jsp     |   48 +
 .../WEB-INF/jsp/threshold-edit-dialog.jsp       |   45 +
 .../WEB-INF/jsp/threshold-filter-dialog.jsp     |   40 +
 .../ui/src/main/webapp/WEB-INF/web.xml          |   48 +
 .../ui/src/main/webapp/css/threshold_styles.css |  422 ++
 .../src/main/webapp/images/addWorksheetRow.png  |  Bin 0 -> 323 bytes
 .../ui/src/main/webapp/images/clear.png         |  Bin 0 -> 912 bytes
 .../ui/src/main/webapp/images/filter.gif        |  Bin 0 -> 623 bytes
 .../ui/src/main/webapp/images/firstPage.gif     |  Bin 0 -> 577 bytes
 .../main/webapp/images/firstPageDisabled.gif    |  Bin 0 -> 365 bytes
 .../ui/src/main/webapp/images/lastPage.gif      |  Bin 0 -> 579 bytes
 .../src/main/webapp/images/lastPageDisabled.gif |  Bin 0 -> 368 bytes
 .../ui/src/main/webapp/images/nextPage.gif      |  Bin 0 -> 570 bytes
 .../src/main/webapp/images/nextPageDisabled.gif |  Bin 0 -> 362 bytes
 .../ui/src/main/webapp/images/prevPage.gif      |  Bin 0 -> 567 bytes
 .../src/main/webapp/images/prevPageDisabled.gif |  Bin 0 -> 361 bytes
 .../main/webapp/images/removeWorksheetRow.png   |  Bin 0 -> 655 bytes
 .../ui/src/main/webapp/images/separator.gif     |  Bin 0 -> 63 bytes
 .../webapp/js/jquery/jqgrid/css/ui.jqgrid.css   |  171 +
 .../js/jquery/jqgrid/js/i18n/grid.locale-en.js  |  170 +
 .../js/jquery/jqgrid/js/jquery.jqGrid.min.js    |  545 +++
 .../ui/src/main/webapp/js/nf-common.js          |  888 +++++
 .../ui/src/main/xsd/threshold_settings.xsd      |   54 +
 .../nar/pom.xml                                 |   36 +
 .../persistent-provenance-repository/pom.xml    |   58 +
 .../nifi/provenance/IndexConfiguration.java     |  398 ++
 .../PersistentProvenanceRepository.java         | 1892 +++++++++
 .../provenance/RepositoryConfiguration.java     |  299 ++
 .../nifi/provenance/StandardRecordReader.java   |  305 ++
 .../nifi/provenance/StandardRecordWriter.java   |  237 ++
 .../provenance/expiration/ExpirationAction.java |   35 +
 .../expiration/FileRemovalAction.java           |   49 +
 .../provenance/lucene/DeleteIndexAction.java    |  139 +
 .../nifi/provenance/lucene/DocsReader.java      |  133 +
 .../nifi/provenance/lucene/FieldNames.java      |   23 +
 .../nifi/provenance/lucene/IndexSearch.java     |   84 +
 .../nifi/provenance/lucene/IndexingAction.java  |  219 ++
 .../nifi/provenance/lucene/LineageQuery.java    |  108 +
 .../nifi/provenance/lucene/LuceneUtil.java      |  141 +
 .../provenance/rollover/CompressionAction.java  |   59 +
 .../provenance/rollover/RolloverAction.java     |   35 +
 .../provenance/serialization/RecordReader.java  |   31 +
 .../provenance/serialization/RecordReaders.java |   70 +
 .../provenance/serialization/RecordWriter.java  |   85 +
 .../provenance/serialization/RecordWriters.java |   30 +
 ...he.nifi.provenance.ProvenanceEventRepository |   15 +
 .../TestPersistentProvenanceRepository.java     | 1118 ++++++
 .../pom.xml                                     |   42 +
 nifi/nar-bundles/pom.xml                        |  134 +
 .../standard-bundle/jms-processors/pom.xml      |   55 +
 .../apache/nifi/processors/jms/GetJMSQueue.java |   75 +
 .../apache/nifi/processors/jms/GetJMSTopic.java |  359 ++
 .../apache/nifi/processors/jms/JmsConsumer.java |  208 +
 .../org/apache/nifi/processors/jms/PutJMS.java  |  374 ++
 .../nifi/processors/jms/util/JmsFactory.java    |  435 +++
 .../nifi/processors/jms/util/JmsProperties.java |  176 +
 .../jms/util/WrappedMessageConsumer.java        |   77 +
 .../jms/util/WrappedMessageProducer.java        |   77 +
 .../org.apache.nifi.processor.Processor         |   17 +
 .../index.html                                  |  118 +
 .../index.html                                  |  122 +
 .../index.html                                  |  152 +
 .../nifi/processors/jms/GetJMSQueueTest.java    |  141 +
 nifi/nar-bundles/standard-bundle/nar/pom.xml    |   57 +
 nifi/nar-bundles/standard-bundle/pom.xml        |   71 +
 .../standard-ganglia-reporter/pom.xml           |   52 +
 .../ganglia/StandardGangliaReporter.java        |  262 ++
 .../org.apache.nifi.reporting.ReportingTask     |   15 +
 .../index.html                                  |   65 +
 .../standard-prioritizers/pom.xml               |   40 +
 .../prioritizer/FirstInFirstOutPrioritizer.java |   37 +
 .../NewestFlowFileFirstPrioritizer.java         |   37 +
 .../OldestFlowFileFirstPrioritizer.java         |   37 +
 .../PriorityAttributePrioritizer.java           |   87 +
 ...org.apache.nifi.flowfile.FlowFilePrioritizer |   18 +
 .../prioritizer/NewestFirstPrioritizerTest.java |   66 +
 .../prioritizer/OldestFirstPrioritizerTest.java |   66 +
 .../PriorityAttributePrioritizerTest.java       |  117 +
 .../standard-bundle/standard-processors/pom.xml |  154 +
 .../standard/Base64EncodeContent.java           |  142 +
 .../processors/standard/CompressContent.java    |  307 ++
 .../nifi/processors/standard/ControlRate.java   |  381 ++
 .../standard/ConvertCharacterSet.java           |  175 +
 .../processors/standard/DetectDuplicate.java    |  249 ++
 .../processors/standard/DistributeLoad.java     |  498 +++
 .../processors/standard/EncryptContent.java     |  263 ++
 .../standard/EvaluateRegularExpression.java     |  294 ++
 .../nifi/processors/standard/EvaluateXPath.java |  429 +++
 .../processors/standard/EvaluateXQuery.java     |  463 +++
 .../standard/ExecuteStreamCommand.java          |  358 ++
 .../processors/standard/GenerateFlowFile.java   |  164 +
 .../apache/nifi/processors/standard/GetFTP.java |   72 +
 .../nifi/processors/standard/GetFile.java       |  456 +++
 .../processors/standard/GetFileTransfer.java    |  301 ++
 .../nifi/processors/standard/GetHTTP.java       |  480 +++
 .../nifi/processors/standard/GetSFTP.java       |   92 +
 .../nifi/processors/standard/HashAttribute.java |  255 ++
 .../nifi/processors/standard/HashContent.java   |  151 +
 .../processors/standard/IdentifyMimeType.java   |  474 +++
 .../nifi/processors/standard/InvokeHTTP.java    |  758 ++++
 .../nifi/processors/standard/ListenHTTP.java    |  321 ++
 .../nifi/processors/standard/ListenUDP.java     |  627 +++
 .../nifi/processors/standard/LogAttribute.java  |  262 ++
 .../nifi/processors/standard/MergeContent.java  | 1015 +++++
 .../nifi/processors/standard/ModifyBytes.java   |  134 +
 .../processors/standard/MonitorActivity.java    |  206 +
 .../nifi/processors/standard/PostHTTP.java      |  904 +++++
 .../nifi/processors/standard/PutEmail.java      |  297 ++
 .../apache/nifi/processors/standard/PutFTP.java |  141 +
 .../nifi/processors/standard/PutFile.java       |  367 ++
 .../processors/standard/PutFileTransfer.java    |  293 ++
 .../nifi/processors/standard/PutSFTP.java       |   85 +
 .../nifi/processors/standard/ReplaceText.java   |  289 ++
 .../standard/ReplaceTextWithMapping.java        |  383 ++
 .../processors/standard/RouteOnAttribute.java   |  261 ++
 .../processors/standard/RouteOnContent.java     |  232 ++
 .../nifi/processors/standard/ScanAttribute.java |  229 ++
 .../nifi/processors/standard/ScanContent.java   |  292 ++
 .../processors/standard/SegmentContent.java     |  163 +
 .../nifi/processors/standard/SplitContent.java  |  260 ++
 .../nifi/processors/standard/SplitText.java     |  370 ++
 .../nifi/processors/standard/SplitXml.java      |  300 ++
 .../nifi/processors/standard/TransformXml.java  |  194 +
 .../nifi/processors/standard/UnpackContent.java |  427 +++
 .../nifi/processors/standard/ValidateXml.java   |  147 +
 .../servlets/ContentAcknowledgmentServlet.java  |  136 +
 .../standard/servlets/ListenHTTPServlet.java    |  320 ++
 .../nifi/processors/standard/util/Bin.java      |  168 +
 .../processors/standard/util/BinManager.java    |  241 ++
 .../standard/util/DocumentReaderCallback.java   |   74 +
 .../processors/standard/util/FTPTransfer.java   |  541 +++
 .../nifi/processors/standard/util/FTPUtils.java |  322 ++
 .../nifi/processors/standard/util/FileInfo.java |  167 +
 .../processors/standard/util/FileTransfer.java  |  244 ++
 .../standard/util/FlowFileSessionWrapper.java   |   44 +
 .../standard/util/NLKBufferedReader.java        |  187 +
 .../standard/util/SFTPConnection.java           |   82 +
 .../processors/standard/util/SFTPTransfer.java  |  637 +++
 .../processors/standard/util/SFTPUtils.java     |  324 ++
 .../standard/util/UDPStreamConsumer.java        |  214 ++
 .../util/ValidatingBase64InputStream.java       |   76 +
 .../standard/util/XmlElementNotifier.java       |   22 +
 .../standard/util/XmlSplitterSaxParser.java     |  112 +
 .../org.apache.nifi.processor.Processor         |   58 +
 .../index.html                                  |   63 +
 .../index.html                                  |  166 +
 .../index.html                                  |  116 +
 .../index.html                                  |   65 +
 .../index.html                                  |  147 +
 .../index.html                                  |  106 +
 .../index.html                                  |   97 +
 .../index.html                                  |  160 +
 .../index.html                                  |  135 +
 .../index.html                                  |  311 ++
 .../index.html                                  |  111 +
 .../index.html                                  |   64 +
 .../index.html                                  |  227 ++
 .../index.html                                  |  186 +
 .../index.html                                  |  143 +
 .../index.html                                  |  250 ++
 .../index.html                                  |   88 +
 .../index.html                                  |   89 +
 .../index.html                                  |  136 +
 .../index.html                                  |  181 +
 .../index.html                                  |   86 +
 .../index.html                                  |  144 +
 .../index.html                                  |   80 +
 .../index.html                                  |  347 ++
 .../index.html                                  |   64 +
 .../index.html                                  |  143 +
 .../index.html                                  |  187 +
 .../index.html                                  |  114 +
 .../index.html                                  |  283 ++
 .../index.html                                  |  109 +
 .../index.html                                  |  281 ++
 .../index.html                                  |   91 +
 .../index.html                                  |  114 +
 .../index.html                                  |  110 +
 .../index.html                                  |   82 +
 .../index.html                                  |   85 +
 .../index.html                                  |  100 +
 .../index.html                                  |  123 +
 .../index.html                                  |  107 +
 .../index.html                                  |  121 +
 .../index.html                                  |   64 +
 .../index.html                                  |   63 +
 .../index.html                                  |  163 +
 .../index.html                                  |   56 +
 .../src/test/java/TestIngestAndUpdate.java      |   34 +
 .../src/test/java/TestSuccess.java              |   24 +
 .../processors/standard/HelloWorldServlet.java  |   41 +
 .../standard/RESTServiceContentModified.java    |   78 +
 .../standard/TestBase64EncodeContent.java       |   83 +
 .../standard/TestCompressContent.java           |  111 +
 .../processors/standard/TestControlRate.java    |   68 +
 .../standard/TestConvertCharacterSet.java       |   47 +
 .../standard/TestDetectDuplicate.java           |  206 +
 .../processors/standard/TestDistributeLoad.java |  138 +
 .../processors/standard/TestEncryptContent.java |   65 +
 .../standard/TestEvaluateRegularExpression.java |  319 ++
 .../processors/standard/TestEvaluateXPath.java  |  159 +
 .../processors/standard/TestEvaluateXQuery.java |  651 ++++
 .../standard/TestExecuteStreamCommand.java      |  188 +
 .../nifi/processors/standard/TestGetFile.java   |  186 +
 .../nifi/processors/standard/TestGetHTTP.java   |  354 ++
 .../processors/standard/TestHashAttribute.java  |   99 +
 .../processors/standard/TestHashContent.java    |   67 +
 .../standard/TestIdentifyMimeType.java          |  131 +
 .../processors/standard/TestInvokeHTTP.java     |  593 +++
 .../nifi/processors/standard/TestListenUDP.java |  214 ++
 .../processors/standard/TestMergeContent.java   |  593 +++
 .../processors/standard/TestModifyBytes.java    |  190 +
 .../standard/TestMonitorActivity.java           |  193 +
 .../processors/standard/TestReplaceText.java    |  371 ++
 .../standard/TestReplaceTextLineByLine.java     |  337 ++
 .../standard/TestReplaceTextWithMapping.java    |  332 ++
 .../standard/TestRouteOnAttribute.java          |  161 +
 .../processors/standard/TestRouteOnContent.java |   73 +
 .../processors/standard/TestScanAttribute.java  |  149 +
 .../processors/standard/TestScanContent.java    |   85 +
 .../processors/standard/TestSegmentContent.java |   65 +
 .../nifi/processors/standard/TestServer.java    |  177 +
 .../processors/standard/TestSplitContent.java   |  235 ++
 .../nifi/processors/standard/TestSplitText.java |  173 +
 .../nifi/processors/standard/TestSplitXml.java  |  103 +
 .../processors/standard/TestTransformXml.java   |  120 +
 .../processors/standard/TestUnpackContent.java  |  226 ++
 .../processors/standard/TestValidateXml.java    |   42 +
 .../standard/UserAgentTestingServlet.java       |   42 +
 .../CharacterSetConversionSamples/Converted.txt |    1 +
 .../Converted2.txt                              |  Bin 0 -> 134580 bytes
 .../CharacterSetConversionSamples/Original.txt  |  418 ++
 .../resources/CompressedData/SampleFile.txt     |  418 ++
 .../resources/CompressedData/SampleFile.txt.bz2 |  Bin 0 -> 388 bytes
 .../resources/CompressedData/SampleFile.txt.gz  |  Bin 0 -> 321 bytes
 .../CompressedData/SampleFile1.txt.bz2          |  Bin 0 -> 154 bytes
 .../resources/CompressedData/SampleFile1.txt.gz |  Bin 0 -> 321 bytes
 .../CompressedData/SampleFileConcat.txt         |  835 ++++
 .../CompressedData/SampleFileConcat.txt.bz2     |  Bin 0 -> 542 bytes
 .../test/resources/ExecuteCommand/1000bytes.txt |    1 +
 .../ExecuteCommand/TestIngestAndUpdate.jar      |  Bin 0 -> 1170 bytes
 .../resources/ExecuteCommand/TestSuccess.jar    |  Bin 0 -> 827 bytes
 .../src/test/resources/ExecuteCommand/test.txt  |    2 +
 .../dictionary-with-empty-new-lines             |    5 +
 .../ScanAttribute/dictionary-with-extra-info    |    9 +
 .../test/resources/ScanAttribute/dictionary1    |    5 +
 .../test/resources/TestIdentifyMimeType/1.7z    |  Bin 0 -> 133 bytes
 .../test/resources/TestIdentifyMimeType/1.jar   |  Bin 0 -> 466 bytes
 .../test/resources/TestIdentifyMimeType/1.mdb   |  Bin 0 -> 237568 bytes
 .../test/resources/TestIdentifyMimeType/1.pdf   |  Bin 0 -> 277032 bytes
 .../test/resources/TestIdentifyMimeType/1.tar   |  Bin 0 -> 2048 bytes
 .../resources/TestIdentifyMimeType/1.txt.bz2    |  Bin 0 -> 59 bytes
 .../resources/TestIdentifyMimeType/1.txt.gz     |  Bin 0 -> 50 bytes
 .../test/resources/TestIdentifyMimeType/1.xml   |   20 +
 .../test/resources/TestIdentifyMimeType/1.zip   |  Bin 0 -> 165 bytes
 .../TestIdentifyMimeType/bgBannerFoot.png       |  Bin 0 -> 189 bytes
 .../TestIdentifyMimeType/blueBtnBg.jpg          |  Bin 0 -> 356 bytes
 .../TestIdentifyMimeType/flowfilev1.tar         |  Bin 0 -> 10240 bytes
 .../resources/TestIdentifyMimeType/flowfilev3   |  Bin 0 -> 40 bytes
 .../resources/TestIdentifyMimeType/grid.gif     |  Bin 0 -> 135 bytes
 .../test/resources/TestMergeContent/demarcate   |    1 +
 .../src/test/resources/TestMergeContent/foot    |    1 +
 .../src/test/resources/TestMergeContent/head    |    1 +
 .../test/resources/TestModifyBytes/noFooter.txt |   10 +
 .../TestModifyBytes/noFooter_noHeader.txt       |   10 +
 .../test/resources/TestModifyBytes/noHeader.txt |   11 +
 .../test/resources/TestModifyBytes/testFile.txt |   11 +
 .../TestReplaceTextLineByLine/$1$1.txt          |   11 +
 .../BRue_cRue_RiRey.txt                         |   11 +
 .../TestReplaceTextLineByLine/Blu$2e_clu$2e.txt |   11 +
 .../TestReplaceTextLineByLine/D$d_h$d.txt       |   11 +
 .../TestReplaceTextLineByLine/Good.txt          |    1 +
 .../TestReplaceTextLineByLine/Spider.txt        |   11 +
 .../TestReplaceTextLineByLine/[DODO].txt        |   11 +
 .../TestReplaceTextLineByLine/cu[$1]_Po[$1].txt |   11 +
 .../TestReplaceTextLineByLine/cu_Po.txt         |   11 +
 .../TestReplaceTextLineByLine/food.txt          |   11 +
 .../TestReplaceTextLineByLine/testFile.txt      |   11 +
 .../color-fruit-backreference-mapping.txt       |    7 +
 .../color-fruit-blank-mapping.txt               |    7 +
 .../color-fruit-escaped-dollar-mapping.txt      |    7 +
 ...t-excessive-backreference-mapping-simple.txt |    6 +
 ...or-fruit-excessive-backreference-mapping.txt |    6 +
 ...olor-fruit-invalid-backreference-mapping.txt |    7 +
 .../color-fruit-mapping.txt                     |    7 +
 .../color-fruit-no-match-mapping.txt            |    7 +
 .../color-fruit-space-mapping.txt               |    6 +
 .../colors-without-dashes.txt                   |    4 +
 .../TestReplaceTextWithMapping/colors.txt       |    4 +
 .../test/resources/TestScanContent/helloWorld   |    1 +
 .../resources/TestScanContent/wellthengood-bye  |    1 +
 .../src/test/resources/TestSplitText/1.txt      |    5 +
 .../src/test/resources/TestSplitText/2.txt      |    5 +
 .../src/test/resources/TestSplitText/3.txt      |    5 +
 .../src/test/resources/TestSplitText/4.txt      |    3 +
 .../src/test/resources/TestSplitText/5.txt      |    7 +
 .../src/test/resources/TestSplitText/6.txt      |    7 +
 .../test/resources/TestSplitText/original.txt   |   12 +
 .../test/resources/TestTransformXml/math.html   |    8 +
 .../test/resources/TestTransformXml/math.xml    |   21 +
 .../test/resources/TestTransformXml/math.xsl    |   36 +
 .../test/resources/TestTransformXml/tokens.csv  |    2 +
 .../test/resources/TestTransformXml/tokens.xml  |   17 +
 .../test/resources/TestTransformXml/tokens.xsl  |  103 +
 .../resources/TestUnpackContent/data.flowfilev2 |  Bin 0 -> 255 bytes
 .../resources/TestUnpackContent/data.flowfilev3 |  Bin 0 -> 357 bytes
 .../test/resources/TestUnpackContent/data.tar   |  Bin 0 -> 3584 bytes
 .../test/resources/TestUnpackContent/data.zip   |  Bin 0 -> 359 bytes
 .../resources/TestUnpackContent/folder/cal.txt  |    8 +
 .../resources/TestUnpackContent/folder/date.txt |    1 +
 .../src/test/resources/TestXml/XmlBundle.xsd    |   34 +
 .../src/test/resources/TestXml/fruit.xml        |   47 +
 .../src/test/resources/TestXml/subNode.xml      |   21 +
 .../src/test/resources/TestXml/xml-bundle-1     |   51 +
 .../src/test/resources/TestXml/xml-snippet.xml  |   25 +
 .../src/test/resources/hello.txt                |    1 +
 .../src/test/resources/localhost-ks.jks         |  Bin 0 -> 1298 bytes
 .../src/test/resources/localhost-ts.jks         |  Bin 0 -> 891 bytes
 .../src/test/resources/localhost.cer            |  Bin 0 -> 829 bytes
 .../src/test/resources/logback-test.xml         |   49 +
 .../standard-reporting-tasks/pom.xml            |   60 +
 .../ControllerStatusReportingTask.java          |  347 ++
 .../nifi/controller/MonitorDiskUsage.java       |  113 +
 .../apache/nifi/controller/MonitorMemory.java   |  260 ++
 .../org.apache.nifi.reporting.ReportingTask     |   17 +
 .../index.html                                  |   85 +
 .../index.html                                  |   58 +
 .../index.html                                  |   77 +
 .../nifi/controller/TestMonitorDiskUsage.java   |   61 +
 .../pom.xml                                     |   36 +
 .../distributed/cache/client/Deserializer.java  |   41 +
 .../cache/client/DistributedMapCacheClient.java |  114 +
 .../cache/client/DistributedSetCacheClient.java |   74 +
 .../distributed/cache/client/Serializer.java    |   40 +
 .../exception/DeserializationException.java     |   33 +
 .../exception/SerializationException.java       |   33 +
 .../distributed-cache-client-service/pom.xml    |   56 +
 .../distributed/cache/client/CommsSession.java  |   46 +
 .../DistributedMapCacheClientService.java       |  305 ++
 .../DistributedSetCacheClientService.java       |  215 ++
 .../cache/client/SSLCommsSession.java           |  106 +
 .../cache/client/StandardCommsSession.java      |  124 +
 ...org.apache.nifi.controller.ControllerService |   16 +
 .../index.html                                  |   78 +
 .../index.html                                  |   51 +
 .../distributed-cache-protocol/pom.xml          |   39 +
 .../cache/protocol/ProtocolHandshake.java       |  119 +
 .../protocol/exception/HandshakeException.java  |   27 +
 .../distributed-cache-server/pom.xml            |   75 +
 .../cache/server/AbstractCacheServer.java       |  199 +
 .../distributed/cache/server/CacheRecord.java   |   57 +
 .../distributed/cache/server/CacheServer.java   |   26 +
 .../cache/server/DistributedCacheServer.java    |  107 +
 .../cache/server/DistributedSetCacheServer.java |   68 +
 .../cache/server/EvictionPolicy.java            |   73 +
 .../cache/server/SetCacheServer.java            |  104 +
 .../server/map/DistributedMapCacheServer.java   |   71 +
 .../distributed/cache/server/map/MapCache.java  |   29 +
 .../cache/server/map/MapCacheRecord.java        |   58 +
 .../cache/server/map/MapCacheServer.java        |  145 +
 .../cache/server/map/MapPutResult.java          |   59 +
 .../cache/server/map/PersistentMapCache.java    |  210 +
 .../cache/server/map/SimpleMapCache.java        |  165 +
 .../cache/server/set/PersistentSetCache.java    |  194 +
 .../distributed/cache/server/set/SetCache.java  |   29 +
 .../cache/server/set/SetCacheRecord.java        |   55 +
 .../cache/server/set/SetCacheResult.java        |   43 +
 .../cache/server/set/SimpleSetCache.java        |  117 +
 ...org.apache.nifi.controller.ControllerService |   16 +
 .../index.html                                  |   82 +
 .../cache/server/TestServerAndClient.java       |  535 +++
 .../src/test/resources/localhost-ks.jks         |  Bin 0 -> 1298 bytes
 .../src/test/resources/localhost-ts.jks         |  Bin 0 -> 891 bytes
 .../distributed-cache-services-nar/pom.xml      |   47 +
 .../distributed-cache-services-bundle/pom.xml   |   35 +
 .../load-distribution-service-api/pom.xml       |   37 +
 .../nifi/loading/LoadDistributionListener.java  |   24 +
 .../nifi/loading/LoadDistributionService.java   |   33 +
 nifi/nar-bundles/standard-services/pom.xml      |   39 +
 .../ssl-context-bundle/nar/pom.xml              |   39 +
 .../ssl-context-bundle/pom.xml                  |   34 +
 .../ssl-context-service/pom.xml                 |   52 +
 .../nifi/ssl/StandardSSLContextService.java     |  354 ++
 ...org.apache.nifi.controller.ControllerService |   15 +
 .../index.html                                  |   63 +
 .../apache/nifi/ssl/SSLContextServiceTest.java  |  197 +
 .../java/org/apache/nifi/ssl/TestProcessor.java |   47 +
 .../src/test/resources/localhost-ks.jks         |  Bin 0 -> 1298 bytes
 .../src/test/resources/localhost-ts.jks         |  Bin 0 -> 891 bytes
 .../ssl-context-service-api/pom.xml             |   36 +
 .../org/apache/nifi/ssl/SSLContextService.java  |   55 +
 .../standard-services-api-nar/pom.xml           |   45 +
 .../update-attribute-bundle/model/pom.xml       |   27 +
 .../apache/nifi/update/attributes/Action.java   |   55 +
 .../nifi/update/attributes/Condition.java       |   49 +
 .../apache/nifi/update/attributes/Criteria.java |  141 +
 .../nifi/update/attributes/FlowFilePolicy.java  |   34 +
 .../org/apache/nifi/update/attributes/Rule.java |   65 +
 .../update/attributes/serde/CriteriaSerDe.java  |  128 +
 .../update-attribute-bundle/nar/pom.xml         |   48 +
 .../nar-bundles/update-attribute-bundle/pom.xml |   56 +
 .../update-attribute-bundle/processor/pom.xml   |   54 +
 .../processors/attributes/UpdateAttribute.java  |  508 +++
 .../org.apache.nifi.processor.Processor         |   15 +
 .../index.html                                  |  253 ++
 .../update/attributes/TestUpdateAttribute.java  |  425 ++
 .../ui/nb-configuration.xml                     |   18 +
 .../update-attribute-bundle/ui/pom.xml          |   91 +
 .../attributes/UpdateAttributeModelFactory.java |  127 +
 .../attributes/api/ObjectMapperResolver.java    |   51 +
 .../update/attributes/api/RuleResource.java     |  684 ++++
 .../nifi/update/attributes/dto/ActionDTO.java   |   68 +
 .../update/attributes/dto/ConditionDTO.java     |   59 +
 .../nifi/update/attributes/dto/DtoFactory.java  |   68 +
 .../nifi/update/attributes/dto/RuleDTO.java     |   65 +
 .../update/attributes/entity/ActionEntity.java  |   65 +
 .../attributes/entity/ConditionEntity.java      |   65 +
 .../entity/EvaluationContextEntity.java         |   74 +
 .../update/attributes/entity/RuleEntity.java    |   65 +
 .../update/attributes/entity/RulesEntity.java   |   66 +
 .../ui/src/main/webapp/META-INF/nifi-processor  |   15 +
 .../src/main/webapp/WEB-INF/jsp/worksheet.jsp   |  188 +
 .../ui/src/main/webapp/WEB-INF/web.xml          |   46 +
 .../ui/src/main/webapp/css/main.css             |  426 +++
 .../ui/src/main/webapp/images/bgInputText.png   |  Bin 0 -> 139 bytes
 .../ui/src/main/webapp/images/buttonNew.png     |  Bin 0 -> 590 bytes
 .../ui/src/main/webapp/images/iconDelete.png    |  Bin 0 -> 670 bytes
 .../ui/src/main/webapp/images/iconInfo.png      |  Bin 0 -> 550 bytes
 .../ui/src/main/webapp/js/application.js        | 1915 +++++++++
 .../nar/pom.xml                                 |   36 +
 .../pom.xml                                     |   44 +
 .../volatile-provenance-repository/pom.xml      |   47 +
 .../VolatileProvenanceRepository.java           |  745 ++++
 ...he.nifi.provenance.ProvenanceEventRepository |   15 +
 .../TestVolatileProvenanceRepository.java       |  178 +
 .../src/test/resources/nifi.properties          |   18 +
 nifi/nifi-api/.gitignore                        |    2 +
 nifi/nifi-api/pom.xml                           |   31 +
 .../apache/nifi/authorization/Authority.java    |   97 +
 .../nifi/authorization/AuthorityProvider.java   |  181 +
 .../AuthorityProviderConfigurationContext.java  |   52 +
 .../AuthorityProviderInitializationContext.java |   27 +
 .../authorization/AuthorityProviderLookup.java  |   25 +
 .../authorization/DownloadAuthorization.java    |   86 +
 .../annotation/AuthorityProviderContext.java    |   36 +
 .../exception/AuthorityAccessException.java     |   33 +
 .../IdentityAlreadyExistsException.java         |   32 +
 .../exception/ProviderCreationException.java    |   40 +
 .../exception/ProviderDestructionException.java |   40 +
 .../exception/UnknownIdentityException.java     |   32 +
 .../AbstractConfigurableComponent.java          |  221 ++
 .../apache/nifi/components/AllowableValue.java  |  136 +
 .../nifi/components/ConfigurableComponent.java  |   77 +
 .../nifi/components/PropertyDescriptor.java     |  552 +++
 .../apache/nifi/components/PropertyValue.java   |  189 +
 .../nifi/components/ValidationContext.java      |   82 +
 .../nifi/components/ValidationResult.java       |  173 +
 .../org/apache/nifi/components/Validator.java   |   55 +
 .../controller/AbstractControllerService.java   |   91 +
 .../nifi/controller/ConfigurationContext.java   |   46 +
 .../nifi/controller/ControllerService.java      |  182 +
 .../ControllerServiceInitializationContext.java |   36 +
 .../controller/ControllerServiceLookup.java     |   66 +
 .../apache/nifi/controller/FlowFileQueue.java   |  200 +
 .../apache/nifi/controller/ScheduledState.java  |   37 +
 .../org/apache/nifi/controller/Snippet.java     |  113 +
 .../org/apache/nifi/controller/Triggerable.java |  116 +
 .../controller/annotation/OnConfigured.java     |   41 +
 .../repository/ContentRepository.java           |  314 ++
 .../controller/repository/FlowFileRecord.java   |   54 +
 .../repository/FlowFileRepository.java          |  128 +
 .../repository/FlowFileSwapManager.java         |   63 +
 .../controller/repository/QueueProvider.java    |   35 +
 .../controller/repository/RepositoryRecord.java |  100 +
 .../repository/RepositoryRecordType.java        |   26 +
 .../repository/claim/ContentClaim.java          |   55 +
 .../repository/claim/ContentClaimManager.java   |  143 +
 .../controller/status/ConnectionStatus.java     |  193 +
 .../nifi/controller/status/PortStatus.java      |  201 +
 .../controller/status/ProcessGroupStatus.java   |  547 +++
 .../nifi/controller/status/ProcessorStatus.java |  275 ++
 .../status/RemoteProcessGroupStatus.java        |  217 ++
 .../nifi/controller/status/RunStatus.java       |   28 +
 .../controller/status/TransmissionStatus.java   |   23 +
 .../history/ComponentStatusRepository.java      |  167 +
 .../status/history/MetricDescriptor.java        |   75 +
 .../status/history/StatusHistory.java           |   50 +
 .../status/history/StatusSnapshot.java          |   48 +
 .../controller/status/history/ValueMapper.java  |   23 +
 .../controller/status/history/ValueReducer.java |   25 +
 .../org/apache/nifi/events/EventReporter.java   |   27 +
 .../nifi/expression/AttributeExpression.java    |   79 +
 .../expression/AttributeValueDecorator.java     |   29 +
 .../expression/ExpressionLanguageCompiler.java  |   69 +
 .../java/org/apache/nifi/flowfile/FlowFile.java |  109 +
 .../nifi/flowfile/FlowFilePrioritizer.java      |   30 +
 .../java/org/apache/nifi/logging/LogLevel.java  |   27 +
 .../org/apache/nifi/logging/ProcessorLog.java   |   73 +
 .../nifi/processor/AbstractProcessor.java       |   37 +
 .../AbstractSessionFactoryProcessor.java        |  122 +
 .../org/apache/nifi/processor/DataUnit.java     |  248 ++
 .../apache/nifi/processor/FlowFileFilter.java   |   91 +
 .../apache/nifi/processor/ProcessContext.java   |  124 +
 .../apache/nifi/processor/ProcessSession.java   |  719 ++++
 .../nifi/processor/ProcessSessionFactory.java   |   30 +
 .../org/apache/nifi/processor/Processor.java    |   91 +
 .../ProcessorInitializationContext.java         |   53 +
 .../org/apache/nifi/processor/QueueSize.java    |   49 +
 .../org/apache/nifi/processor/Relationship.java |  127 +
 .../nifi/processor/SchedulingContext.java       |   70 +
 .../annotation/CapabilityDescription.java       |   39 +
 .../nifi/processor/annotation/EventDriven.java  |   49 +
 .../nifi/processor/annotation/OnAdded.java      |   41 +
 .../nifi/processor/annotation/OnRemoved.java    |   42 +
 .../nifi/processor/annotation/OnScheduled.java  |   44 +
 .../nifi/processor/annotation/OnShutdown.java   |   38 +
 .../nifi/processor/annotation/OnStopped.java    |   56 +
 .../processor/annotation/OnUnscheduled.java     |   46 +
 .../processor/annotation/SideEffectFree.java    |   45 +
 .../processor/annotation/SupportsBatching.java  |   51 +
 .../apache/nifi/processor/annotation/Tags.java  |   44 +
 .../processor/annotation/TriggerSerially.java   |   39 +
 .../TriggerWhenAnyDestinationAvailable.java     |   40 +
 .../processor/annotation/TriggerWhenEmpty.java  |   41 +
 .../exception/FlowFileAccessException.java      |   37 +
 .../exception/FlowFileHandlingException.java    |   39 +
 .../exception/MissingFlowFileException.java     |   37 +
 .../processor/exception/ProcessException.java   |   44 +
 .../nifi/processor/io/InputStreamCallback.java  |   38 +
 .../nifi/processor/io/OutputStreamCallback.java |   39 +
 .../nifi/processor/io/StreamCallback.java       |   41 +
 .../nifi/provenance/ProvenanceEventBuilder.java |  308 ++
 .../nifi/provenance/ProvenanceEventRecord.java  |  296 ++
 .../provenance/ProvenanceEventRepository.java   |  190 +
 .../nifi/provenance/ProvenanceEventType.java    |   92 +
 .../nifi/provenance/ProvenanceReporter.java     |  616 +++
 .../lineage/ComputeLineageResult.java           |   70 +
 .../lineage/ComputeLineageSubmission.java       |   82 +
 .../apache/nifi/provenance/lineage/Lineage.java |   41 +
 .../lineage/LineageComputationType.java         |   27 +
 .../nifi/provenance/lineage/LineageEdge.java    |   26 +
 .../nifi/provenance/lineage/LineageNode.java    |   61 +
 .../provenance/lineage/LineageNodeType.java     |   23 +
 .../lineage/ProvenanceEventLineageNode.java     |   32 +
 .../apache/nifi/provenance/search/Query.java    |   91 +
 .../nifi/provenance/search/QueryResult.java     |   77 +
 .../nifi/provenance/search/QuerySubmission.java |   63 +
 .../nifi/provenance/search/SearchTerm.java      |   24 +
 .../nifi/provenance/search/SearchTerms.java     |   39 +
 .../nifi/provenance/search/SearchableField.java |   62 +
 .../provenance/search/SearchableFieldType.java  |   28 +
 .../nifi/reporting/AbstractReportingTask.java   |   94 +
 .../org/apache/nifi/reporting/Bulletin.java     |  120 +
 .../apache/nifi/reporting/BulletinQuery.java    |  109 +
 .../nifi/reporting/BulletinRepository.java      |   88 +
 .../org/apache/nifi/reporting/EventAccess.java  |   53 +
 .../nifi/reporting/InitializationException.java |   32 +
 .../apache/nifi/reporting/ReportingContext.java |   99 +
 .../ReportingInitializationContext.java         |   80 +
 .../apache/nifi/reporting/ReportingTask.java    |   75 +
 .../org/apache/nifi/reporting/Severity.java     |   24 +
 .../nifi/scheduling/SchedulingStrategy.java     |   86 +
 .../org/apache/nifi/search/SearchContext.java   |   57 +
 .../org/apache/nifi/search/SearchResult.java    |   82 +
 .../java/org/apache/nifi/search/Searchable.java |   27 +
 .../nifi/web/ClusterRequestException.java       |   38 +
 .../nifi/web/InvalidRevisionException.java      |   33 +
 .../org/apache/nifi/web/NiFiWebContext.java     |  121 +
 .../apache/nifi/web/NiFiWebContextConfig.java   |   54 +
 .../nifi/web/ProcessorConfigurationAction.java  |  137 +
 .../java/org/apache/nifi/web/ProcessorInfo.java |  110 +
 .../nifi/web/ResourceNotFoundException.java     |   32 +
 .../main/java/org/apache/nifi/web/Revision.java |  110 +
 .../nifi/components/TestPropertyDescriptor.java |   59 +
 .../org/apache/nifi/processor/TestDataUnit.java |   44 +
 .../src/test/resources/logback-test.xml         |   27 +
 nifi/nifi-bootstrap/pom.xml                     |   29 +
 .../apache/nifi/bootstrap/BootstrapCodec.java   |   92 +
 .../org/apache/nifi/bootstrap/NiFiListener.java |  128 +
 .../java/org/apache/nifi/bootstrap/RunNiFi.java |  941 +++++
 .../org/apache/nifi/bootstrap/ShutdownHook.java |   86 +
 .../exception/InvalidCommandException.java      |   37 +
 .../bootstrap/util/LimitingInputStream.java     |  107 +
 nifi/nifi-docs/pom.xml                          |  131 +
 .../src/main/asciidoc/administration-guide.adoc |  160 +
 .../src/main/asciidoc/developer-guide.adoc      |   45 +
 .../asciidoc/expression-language-guide.adoc     | 1727 +++++++++
 .../images/add-processor-with-tag-cloud.png     |  Bin 0 -> 22925 bytes
 .../src/main/asciidoc/images/add-processor.png  |  Bin 0 -> 31524 bytes
 .../src/main/asciidoc/images/addConnect.png     |  Bin 0 -> 1996 bytes
 .../src/main/asciidoc/images/comments-tab.png   |  Bin 0 -> 6431 bytes
 .../src/main/asciidoc/images/components.png     |  Bin 0 -> 8992 bytes
 .../asciidoc/images/connection-settings.png     |  Bin 0 -> 15214 bytes
 .../main/asciidoc/images/create-connection.png  |  Bin 0 -> 9430 bytes
 .../asciidoc/images/edit-property-dropdown.png  |  Bin 0 -> 14291 bytes
 .../asciidoc/images/edit-property-textarea.png  |  Bin 0 -> 17146 bytes
 .../main/asciidoc/images/event-attributes.png   |  Bin 0 -> 94668 bytes
 .../src/main/asciidoc/images/event-content.png  |  Bin 0 -> 88726 bytes
 .../src/main/asciidoc/images/event-details.png  |  Bin 0 -> 139736 bytes
 .../src/main/asciidoc/images/expand-event.png   |  Bin 0 -> 40476 bytes
 .../main/asciidoc/images/expanded-events.png    |  Bin 0 -> 76082 bytes
 .../src/main/asciidoc/images/find-parents.png   |  Bin 0 -> 35831 bytes
 .../src/main/asciidoc/images/iconAlert.png      |  Bin 0 -> 1396 bytes
 .../src/main/asciidoc/images/iconConnection.png |  Bin 0 -> 1517 bytes
 .../src/main/asciidoc/images/iconDelete.png     |  Bin 0 -> 670 bytes
 .../src/main/asciidoc/images/iconDisable.png    |  Bin 0 -> 764 bytes
 .../src/main/asciidoc/images/iconEdit.png       |  Bin 0 -> 493 bytes
 .../src/main/asciidoc/images/iconEnable.png     |  Bin 0 -> 667 bytes
 .../src/main/asciidoc/images/iconExport.png     |  Bin 0 -> 453 bytes
 .../main/asciidoc/images/iconFlowHistory.png    |  Bin 0 -> 2463 bytes
 .../src/main/asciidoc/images/iconFunnel.png     |  Bin 0 -> 1223 bytes
 .../src/main/asciidoc/images/iconInfo.png       |  Bin 0 -> 550 bytes
 .../src/main/asciidoc/images/iconInputPort.png  |  Bin 0 -> 1842 bytes
 .../main/asciidoc/images/iconInputPortSmall.png |  Bin 0 -> 532 bytes
 .../src/main/asciidoc/images/iconLabel.png      |  Bin 0 -> 838 bytes
 .../src/main/asciidoc/images/iconLineage.png    |  Bin 0 -> 2214 bytes
 .../main/asciidoc/images/iconNewTemplate.png    |  Bin 0 -> 557 bytes
 .../src/main/asciidoc/images/iconNotSecure.png  |  Bin 0 -> 221 bytes
 .../src/main/asciidoc/images/iconOutputPort.png |  Bin 0 -> 1658 bytes
 .../asciidoc/images/iconOutputPortSmall.png     |  Bin 0 -> 459 bytes
 .../main/asciidoc/images/iconProcessGroup.png   |  Bin 0 -> 1422 bytes
 .../src/main/asciidoc/images/iconProcessor.png  |  Bin 0 -> 1446 bytes
 .../src/main/asciidoc/images/iconProvenance.png |  Bin 0 -> 2268 bytes
 .../asciidoc/images/iconRemoteProcessGroup.png  |  Bin 0 -> 674 bytes
 .../src/main/asciidoc/images/iconResize.png     |  Bin 0 -> 165 bytes
 .../src/main/asciidoc/images/iconRun.png        |  Bin 0 -> 538 bytes
 .../src/main/asciidoc/images/iconSecure.png     |  Bin 0 -> 225 bytes
 .../src/main/asciidoc/images/iconSettings.png   |  Bin 0 -> 2638 bytes
 .../src/main/asciidoc/images/iconStop.png       |  Bin 0 -> 402 bytes
 .../src/main/asciidoc/images/iconSummary.png    |  Bin 0 -> 272 bytes
 .../src/main/asciidoc/images/iconTemplate.png   |  Bin 0 -> 970 bytes
 .../asciidoc/images/iconTransmissionActive.png  |  Bin 0 -> 1330 bytes
 .../images/iconTransmissionInactive.png         |  Bin 0 -> 1248 bytes
 .../src/main/asciidoc/images/iconUsers.png      |  Bin 0 -> 2272 bytes
 .../main/asciidoc/images/iconViewDetails.png    |  Bin 0 -> 1788 bytes
 .../images/instantiate-template-description.png |  Bin 0 -> 6508 bytes
 .../asciidoc/images/instantiate-template.png    |  Bin 0 -> 3216 bytes
 .../main/asciidoc/images/invalid-processor.png  |  Bin 0 -> 8187 bytes
 .../main/asciidoc/images/lineage-flowfile.png   |  Bin 0 -> 3855 bytes
 .../asciidoc/images/lineage-graph-annotated.png |  Bin 0 -> 170122 bytes
 .../src/main/asciidoc/images/new-flow.png       |  Bin 0 -> 266913 bytes
 .../main/asciidoc/images/nifi-arch-cluster.png  |  Bin 0 -> 50384 bytes
 .../src/main/asciidoc/images/nifi-arch.png      |  Bin 0 -> 43509 bytes
 .../main/asciidoc/images/nifi-navigation.png    |  Bin 0 -> 339195 bytes
 .../asciidoc/images/nifi-toolbar-components.png |  Bin 0 -> 262314 bytes
 .../src/main/asciidoc/images/parent-found.png   |  Bin 0 -> 42814 bytes
 .../asciidoc/images/process-group-anatomy.png   |  Bin 0 -> 48554 bytes
 .../main/asciidoc/images/processor-anatomy.png  |  Bin 0 -> 37035 bytes
 .../images/processor-connection-bubble.png      |  Bin 0 -> 9936 bytes
 .../src/main/asciidoc/images/properties-tab.png |  Bin 0 -> 11847 bytes
 .../asciidoc/images/provenance-annotated.png    |  Bin 0 -> 530929 bytes
 .../main/asciidoc/images/provenance-table.png   |  Bin 0 -> 466599 bytes
 .../asciidoc/images/remote-group-anatomy.png    |  Bin 0 -> 61383 bytes
 .../images/remote-group-ports-dialog.png        |  Bin 0 -> 21662 bytes
 .../images/remote-port-connection-status.png    |  Bin 0 -> 57332 bytes
 .../src/main/asciidoc/images/scheduling-tab.png |  Bin 0 -> 12288 bytes
 .../src/main/asciidoc/images/search-events.png  |  Bin 0 -> 62626 bytes
 .../images/search-receive-event-abc.png         |  Bin 0 -> 67817 bytes
 .../src/main/asciidoc/images/settings-tab.png   |  Bin 0 -> 16225 bytes
 .../src/main/asciidoc/images/simple-flow.png    |  Bin 0 -> 196808 bytes
 .../src/main/asciidoc/images/stats-history.png  |  Bin 0 -> 33205 bytes
 .../src/main/asciidoc/images/status-bar.png     |  Bin 0 -> 249077 bytes
 .../main/asciidoc/images/summary-annotated.png  |  Bin 0 -> 111956 bytes
 .../src/main/asciidoc/images/summary-table.png  |  Bin 0 -> 62114 bytes
 .../main/asciidoc/images/valid-processor.png    |  Bin 0 -> 8090 bytes
 nifi/nifi-docs/src/main/asciidoc/overview.adoc  |  296 ++
 .../nifi-docs/src/main/asciidoc/user-guide.adoc | 1286 +++++++
 .../src/main/assembly/dependencies.xml          |   28 +
 nifi/nifi-mock/pom.xml                          |   54 +
 .../MockProvenanceEventRepository.java          |  131 +
 .../apache/nifi/reporting/BulletinFactory.java  |   43 +
 .../org/apache/nifi/reporting/MockBulletin.java |   24 +
 .../util/ControllerServiceConfiguration.java    |   74 +
 .../nifi/util/MockBulletinRepository.java       |   74 +
 .../nifi/util/MockConfigurationContext.java     |   50 +
 ...kControllerServiceInitializationContext.java |   41 +
 .../nifi/util/MockControllerServiceLookup.java  |   76 +
 .../org/apache/nifi/util/MockEventAccess.java   |   70 +
 .../java/org/apache/nifi/util/MockFlowFile.java |  279 ++
 .../org/apache/nifi/util/MockFlowFileQueue.java |   85 +
 .../apache/nifi/util/MockProcessContext.java    |  261 ++
 .../apache/nifi/util/MockProcessSession.java    | 1010 +++++
 .../MockProcessorInitializationContext.java     |   74 +
 .../org/apache/nifi/util/MockProcessorLog.java  |  402 ++
 .../org/apache/nifi/util/MockPropertyValue.java |  185 +
 .../nifi/util/MockProvenanceReporter.java       |  202 +
 .../apache/nifi/util/MockReportingContext.java  |  117 +
 .../MockReportingInitializationContext.java     |   81 +
 .../apache/nifi/util/MockSessionFactory.java    |   46 +
 .../apache/nifi/util/MockValidationContext.java |   93 +
 .../org/apache/nifi/util/ReflectionUtils.java   |  149 +
 .../apache/nifi/util/SharedSessionState.java    |   91 +
 .../apache/nifi/util/SingleSessionFactory.java  |   35 +
 .../nifi/util/StandardProcessorTestRunner.java  |  492 +++
 .../java/org/apache/nifi/util/TestRunner.java   |  542 +++
 .../java/org/apache/nifi/util/TestRunners.java  |   37 +
 nifi/pom.xml                                    | 1085 ++++++
 pom.xml                                         | 1092 ------
 4901 files changed, 324930 insertions(+), 324901 deletions(-)
----------------------------------------------------------------------



[19/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeProtocolSender.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeProtocolSender.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeProtocolSender.java
deleted file mode 100644
index 1edcb91..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeProtocolSender.java
+++ /dev/null
@@ -1,73 +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.cluster.protocol;
-
-import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage;
-import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage;
-import org.apache.nifi.cluster.protocol.message.ControllerStartupFailureMessage;
-import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
-import org.apache.nifi.cluster.protocol.message.NodeBulletinsMessage;
-import org.apache.nifi.cluster.protocol.message.ReconnectionFailureMessage;
-
-/**
- * An interface for sending protocol messages from a node to the cluster manager.
- * @author unattributed
- */
-public interface NodeProtocolSender {
-    
-    /**
-     * Sends a "connection request" message to the cluster manager.
-     * @param msg a message
-     * @return the response
-     * @throws UnknownServiceAddressException if the cluster manager's address is not known
-     * @throws ProtocolException if communication failed
-     */
-    ConnectionResponseMessage requestConnection(ConnectionRequestMessage msg) throws ProtocolException, UnknownServiceAddressException;
-    
-    /**
-     * Sends a "heartbeat" message to the cluster manager.
-     * @param msg a message
-     * @throws UnknownServiceAddressException if the cluster manager's address is not known
-     * @throws ProtocolException if communication failed
-     */
-    void heartbeat(HeartbeatMessage msg) throws ProtocolException, UnknownServiceAddressException;
-    
-    /**
-     * Sends a bulletins message to the cluster manager.
-     * @param msg
-     * @throws ProtocolException
-     * @throws UnknownServiceAddressException 
-     */
-    void sendBulletins(NodeBulletinsMessage msg) throws ProtocolException, UnknownServiceAddressException;
-    
-    /**
-     * Sends a failure notification if the controller was unable start.
-     * @param msg a message
-     * @throws UnknownServiceAddressException if the cluster manager's address is not known
-     * @throws ProtocolException if communication failed
-     */
-    void notifyControllerStartupFailure(ControllerStartupFailureMessage msg) throws ProtocolException, UnknownServiceAddressException;
-    
-    /**
-     * Sends a failure notification if the node was unable to reconnect to the cluster
-     * @param msg a message
-     * @throws UnknownServiceAddressException if the cluster manager's address is not known
-     * @throws ProtocolException if communication failed
-     */
-    void notifyReconnectionFailure(ReconnectionFailureMessage msg) throws ProtocolException, UnknownServiceAddressException;
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolContext.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolContext.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolContext.java
deleted file mode 100644
index b614e76..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolContext.java
+++ /dev/null
@@ -1,39 +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.cluster.protocol;
-
-/**
- * The context for communicating using the internal cluster protocol. 
- * 
- * @param <T> The type of protocol message.
- * 
- * @author unattributed
- */
-public interface ProtocolContext<T> {
- 
-    /**
-     * Creates a marshaller for serializing protocol messages.
-     * @return a marshaller
-     */
-    ProtocolMessageMarshaller<T> createMarshaller();
-    
-    /**
-     * Creates an unmarshaller for deserializing protocol messages.
-     * @return a unmarshaller
-     */
-    ProtocolMessageUnmarshaller<T> createUnmarshaller();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolException.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolException.java
deleted file mode 100644
index f11ad84..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolException.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.cluster.protocol;
-
-/**
- * The base exception for problems encountered while communicating within the
- * cluster.
- * @author unattributed
- */
-public class ProtocolException extends RuntimeException {
-    
-    public ProtocolException() {
-    }
-    
-    public ProtocolException(String msg) {
-        super(msg);
-    }
-    
-    public ProtocolException(Throwable cause) {
-        super(cause);
-    }
-    
-    public ProtocolException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolHandler.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolHandler.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolHandler.java
deleted file mode 100644
index 6de87db..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolHandler.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.cluster.protocol;
-
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-
-/**
- * A handler for processing protocol messages.
- * @author unattributed
- */
-public interface ProtocolHandler {
-    
-    /**
-     * Handles the given protocol message or throws an exception if it cannot
-     * handle the message.  If no response is needed by the protocol, then null
-     * should be returned.
-     * 
-     * @param msg a message
-     * @return a response or null, if no response is necessary
-     * 
-     * @throws ProtocolException if the message could not be processed
-     */
-    ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException;
-    
-    /**
-     * @param msg
-     * @return true if the handler can process the given message; false otherwise
-     */
-    boolean canHandle(ProtocolMessage msg);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolListener.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolListener.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolListener.java
deleted file mode 100644
index 32f0f5d..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolListener.java
+++ /dev/null
@@ -1,72 +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.cluster.protocol;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.nifi.reporting.BulletinRepository;
-
-/**
- * Defines the interface for a listener to process protocol messages.
- * @author unattributed
- */
-public interface ProtocolListener {
-    
-    /**
-     * Starts the instance for listening for messages.  Start may only be called
-     * if the instance is not running.
-     * @throws java.io.IOException
-     */
-    void start() throws IOException;
-    
-    /**
-     * Stops the instance from listening for messages.  Stop may only be called
-     * if the instance is running.
-     * @throws java.io.IOException
-     */
-    void stop() throws IOException;
-    
-    /**
-     * @return true if the instance is started; false otherwise.
-     */
-    boolean isRunning();
-    
-    /**
-     * @return the handlers registered with the listener
-     */
-    Collection<ProtocolHandler> getHandlers();
-    
-    /**
-     * Registers a handler with the listener.
-     * @param handler a handler
-     */
-    void addHandler(ProtocolHandler handler);
-    
-    /**
-     * Sets the BulletinRepository that can be used to report bulletins
-     * @param bulletinRepository
-     */
-    void setBulletinRepository(BulletinRepository bulletinRepository);
-    
-    /**
-     * Unregisters the handler with the listener.
-     * @param handler a handler
-     * @return true if the handler was removed; false otherwise
-     */
-    boolean removeHandler(ProtocolHandler handler);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageMarshaller.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageMarshaller.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageMarshaller.java
deleted file mode 100644
index bb436e0..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageMarshaller.java
+++ /dev/null
@@ -1,38 +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.cluster.protocol;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-/**
- * Defines a marshaller for serializing protocol messages.
- * 
- * @param <T> The type of protocol message.
- * 
- * @author unattributed
- */
-public interface ProtocolMessageMarshaller<T> {
-    
-    /**
-     * Serializes the given message to the given output stream.
-     * @param msg a message
-     * @param os an output stream
-     * @throws IOException if the message could not be serialized to the stream
-     */
-    void marshal(T msg, OutputStream os) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageUnmarshaller.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageUnmarshaller.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageUnmarshaller.java
deleted file mode 100644
index c690e7b..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageUnmarshaller.java
+++ /dev/null
@@ -1,38 +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.cluster.protocol;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- * Defines an unmarshaller for deserializing protocol messages.
- * 
- * @param <T> The type of protocol message.
- * 
- * @author unattributed
- */
-public interface ProtocolMessageUnmarshaller<T> {
-    
-    /**
-     * Deserializes a message on the given input stream.
-     * @param is an input stream
-     * @return 
-     * @throws IOException if the message could not be deserialized from the stream
-     */
-    T unmarshal(InputStream is) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/StandardDataFlow.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/StandardDataFlow.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/StandardDataFlow.java
deleted file mode 100644
index c2d16fc..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/StandardDataFlow.java
+++ /dev/null
@@ -1,105 +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.cluster.protocol;
-
-import org.apache.nifi.cluster.protocol.DataFlow;
-import java.io.Serializable;
-import java.util.Arrays;
-
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-
-import org.apache.nifi.cluster.protocol.jaxb.message.DataFlowAdapter;
-
-/**
- * Represents a dataflow, which includes the raw bytes of the flow.xml and 
- * whether processors should be started automatically at application startup.
- */
-@XmlJavaTypeAdapter(DataFlowAdapter.class)
-public class StandardDataFlow implements Serializable, DataFlow {
-    
-    private final byte[] flow;
-    private final byte[] templateBytes;
-    private final byte[] snippetBytes;
-
-    private boolean autoStartProcessors;
-    
-    /**
-     * Constructs an instance.  
-     * 
-     * @param flow a valid flow as bytes, which cannot be null
-     * @param templateBytes an XML representation of templates
-     * @param snippetBytes an XML representation of snippets
-     * 
-     * @throws NullPointerException if any argument is null
-     */
-    public StandardDataFlow(final byte[] flow, final byte[] templateBytes, final byte[] snippetBytes) {
-        this.flow = flow;
-        this.templateBytes = templateBytes;
-        this.snippetBytes = snippetBytes;
-    }
-    
-    public StandardDataFlow(final DataFlow toCopy) {
-        this.flow = copy(toCopy.getFlow());
-        this.templateBytes = copy(toCopy.getTemplates());
-        this.snippetBytes = copy(toCopy.getSnippets());
-        this.autoStartProcessors = toCopy.isAutoStartProcessors();
-    }
-    
-    private static byte[] copy(final byte[] bytes) {
-        return bytes == null ? null : Arrays.copyOf(bytes, bytes.length);
-    }
-    
-    /**
-     * @return the raw byte array of the flow 
-     */
-    public byte[] getFlow() {
-        return flow;
-    }
-
-    /**
-     * @return the raw byte array of the templates
-     */
-    public byte[] getTemplates() {
-        return templateBytes;
-    }
-    
-    /**
-     * @return the raw byte array of the snippets
-     */
-    public byte[] getSnippets() {
-        return snippetBytes;
-    }
-    
-    /**
-     * @return true if processors should be automatically started at application 
-     * startup; false otherwise 
-     */
-    public boolean isAutoStartProcessors() {
-        return autoStartProcessors;
-    }
-    
-    /**
-     * 
-     * Sets the flag to automatically start processors at application startup.
-     * 
-     * @param autoStartProcessors true if processors should be automatically
-     * started at application startup; false otherwise
-     */
-    public void setAutoStartProcessors(final boolean autoStartProcessors) {
-        this.autoStartProcessors = autoStartProcessors;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/UnknownServiceAddressException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/UnknownServiceAddressException.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/UnknownServiceAddressException.java
deleted file mode 100644
index 41c74eb..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/UnknownServiceAddressException.java
+++ /dev/null
@@ -1,39 +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.cluster.protocol;
-
-/**
- * Represents the exceptional case when a service's address is not known.
- * @author unattributed
- */
-public class UnknownServiceAddressException extends RuntimeException {
-    
-    public UnknownServiceAddressException() {
-    }
-    
-    public UnknownServiceAddressException(String msg) {
-        super(msg);
-    }
-    
-    public UnknownServiceAddressException(Throwable cause) {
-        super(cause);
-    }
-    
-    public UnknownServiceAddressException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java
deleted file mode 100644
index ceb3fcb..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java
+++ /dev/null
@@ -1,245 +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.cluster.protocol.impl;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.net.SocketException;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller;
-import org.apache.nifi.cluster.protocol.message.DisconnectMessage;
-import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
-import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
-import org.apache.nifi.cluster.protocol.message.PrimaryRoleAssignmentMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
-import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
-import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage;
-import org.apache.nifi.io.socket.SocketConfiguration;
-import org.apache.nifi.io.socket.SocketUtils;
-import org.apache.nifi.reporting.BulletinRepository;
-import org.apache.nifi.util.FormatUtils;
-
-/**
- * A protocol sender for sending protocol messages from the cluster manager to
- * nodes.  
- * 
- * Connection-type requests (e.g., reconnection, disconnection) by nature of 
- * starting/stopping flow controllers take longer than other types of protocol 
- * messages.  Therefore, a handshake timeout may be specified to lengthen the 
- * allowable time for communication with the node.
- * 
- * @author unattributed
- */
-public class ClusterManagerProtocolSenderImpl implements ClusterManagerProtocolSender {
-
-    
-    private final ProtocolContext<ProtocolMessage> protocolContext;
-    private final SocketConfiguration socketConfiguration;
-    private int handshakeTimeoutSeconds;
-    private volatile BulletinRepository bulletinRepository;
-
-    public ClusterManagerProtocolSenderImpl(final SocketConfiguration socketConfiguration, final ProtocolContext<ProtocolMessage> protocolContext) {
-        if(socketConfiguration == null) {
-            throw new IllegalArgumentException("Socket configuration may not be null.");
-        } else if(protocolContext == null) {
-            throw new IllegalArgumentException("Protocol Context may not be null.");
-        }
-        this.socketConfiguration = socketConfiguration;
-        this.protocolContext = protocolContext;
-        this.handshakeTimeoutSeconds = -1;  // less than zero denotes variable not configured
-    }
-    
-    @Override
-    public void setBulletinRepository(final BulletinRepository bulletinRepository) {
-        this.bulletinRepository = bulletinRepository;
-    }
-
-    /**
-     * Requests the data flow from a node.
-     * @param msg a message
-     * @return the message response
-     * @throws @throws ProtocolException if the message failed to be sent or the response was malformed
-     */
-    @Override
-    public FlowResponseMessage requestFlow(final FlowRequestMessage msg) throws ProtocolException {
-        Socket socket = null;
-        try {
-        	socket = createSocket(msg.getNodeId(), false);
-            
-            try {
-                // marshal message to output stream
-                final ProtocolMessageMarshaller<ProtocolMessage> marshaller = protocolContext.createMarshaller();
-                marshaller.marshal(msg, socket.getOutputStream());
-            } catch(final IOException ioe) {
-                throw new ProtocolException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe);
-            }
-            
-            final ProtocolMessage response;
-            try {
-                // unmarshall response and return
-                final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = protocolContext.createUnmarshaller();
-                response = unmarshaller.unmarshal(socket.getInputStream());
-            } catch(final IOException ioe) {
-                throw new ProtocolException("Failed unmarshalling '" + MessageType.FLOW_RESPONSE + "' protocol message due to: " + ioe, ioe);
-            } 
-            
-            if(MessageType.FLOW_RESPONSE == response.getType()) {
-                return (FlowResponseMessage) response;
-            } else {
-                throw new ProtocolException("Expected message type '" + MessageType.FLOW_RESPONSE + "' but found '" + response.getType() + "'");
-            }
-            
-        } finally {
-            SocketUtils.closeQuietly(socket);
-        }
-    }
-
-    /**
-     * Requests a node to reconnect to the cluster.  The configured value for
-     * handshake timeout is applied to the socket before making the request.
-     * @param msg a message
-     * @return the response
-     * @throws ProtocolException if the message failed to be sent or the response was malformed
-     */
-    @Override
-    public ReconnectionResponseMessage requestReconnection(final ReconnectionRequestMessage msg) throws ProtocolException {
-        Socket socket = null;
-        try {
-        	socket = createSocket(msg.getNodeId(), true);
-
-            // marshal message to output stream
-            try {
-                final ProtocolMessageMarshaller<ProtocolMessage> marshaller = protocolContext.createMarshaller();
-                marshaller.marshal(msg, socket.getOutputStream());
-            } catch(final IOException ioe) {
-                throw new ProtocolException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe);
-            }
-            
-            
-            final ProtocolMessage response;
-            try {
-                // unmarshall response and return
-                final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = protocolContext.createUnmarshaller();
-                response = unmarshaller.unmarshal(socket.getInputStream());
-            } catch(final IOException ioe) {
-                throw new ProtocolException("Failed unmarshalling '" + MessageType.RECONNECTION_RESPONSE + "' protocol message due to: " + ioe, ioe);
-            } 
-            
-            if(MessageType.RECONNECTION_RESPONSE == response.getType()) {
-                return (ReconnectionResponseMessage) response;
-            } else {
-                throw new ProtocolException("Expected message type '" + MessageType.FLOW_RESPONSE + "' but found '" + response.getType() + "'");
-            }
-        } finally {
-            SocketUtils.closeQuietly(socket);
-        }
-    }
-    
-    /**
-     * Requests a node to disconnect from the cluster.  The configured value for
-     * handshake timeout is applied to the socket before making the request.
-     * @param msg a message
-     * @throws ProtocolException if the message failed to be sent
-     */
-    @Override
-    public void disconnect(final DisconnectMessage msg) throws ProtocolException {
-        Socket socket = null;
-        try {
-        	socket = createSocket(msg.getNodeId(), true);
-
-            // marshal message to output stream
-            try {
-                final ProtocolMessageMarshaller<ProtocolMessage> marshaller = protocolContext.createMarshaller();
-                marshaller.marshal(msg, socket.getOutputStream());
-            } catch(final IOException ioe) {
-                throw new ProtocolException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe);
-            }
-        } finally {
-            SocketUtils.closeQuietly(socket);
-        }
-    }
-
-    /**
-     * Assigns the primary role to a node.
-     * 
-     * @param msg a message
-     * 
-     * @throws ProtocolException if the message failed to be sent
-     */
-    @Override
-    public void assignPrimaryRole(final PrimaryRoleAssignmentMessage msg) throws ProtocolException {
-        Socket socket = null;
-        try {
-        	socket = createSocket(msg.getNodeId(), true);
-
-            try {
-                // marshal message to output stream
-                final ProtocolMessageMarshaller<ProtocolMessage> marshaller = protocolContext.createMarshaller();
-                marshaller.marshal(msg, socket.getOutputStream());
-            } catch(final IOException ioe) {
-                throw new ProtocolException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe);
-            }
-        } finally {
-            SocketUtils.closeQuietly(socket);
-        }
-    }
-    
-    
-    private void setConnectionHandshakeTimeoutOnSocket(final Socket socket) throws SocketException {
-        // update socket timeout, if handshake timeout was set; otherwise use socket's current timeout
-        if(handshakeTimeoutSeconds >= 0) {
-            socket.setSoTimeout(handshakeTimeoutSeconds * 1000);
-        }   
-    }
-    
-    public SocketConfiguration getSocketConfiguration() {
-        return socketConfiguration;
-    }
-
-    public int getHandshakeTimeoutSeconds() {
-        return handshakeTimeoutSeconds;
-    }
-
-    public void setHandshakeTimeout(final String handshakeTimeout) {
-        this.handshakeTimeoutSeconds = (int) FormatUtils.getTimeDuration(handshakeTimeout, TimeUnit.SECONDS);
-    }
-
-    private Socket createSocket(final NodeIdentifier nodeId, final boolean applyHandshakeTimeout) {
-    	return createSocket(nodeId.getSocketAddress(), nodeId.getSocketPort(), applyHandshakeTimeout);
-    }
-    
-    private Socket createSocket(final String host, final int port, final boolean applyHandshakeTimeout) {
-    	try {
-            // create a socket
-            final Socket socket = SocketUtils.createSocket(InetSocketAddress.createUnresolved(host, port), socketConfiguration);
-            if ( applyHandshakeTimeout ) {
-            	setConnectionHandshakeTimeoutOnSocket(socket);
-            }
-            return socket;
-        } catch(final IOException ioe) {
-            throw new ProtocolException("Failed to create socket due to: " + ioe, ioe);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java
deleted file mode 100644
index 933e5fa..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java
+++ /dev/null
@@ -1,118 +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.cluster.protocol.impl;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.ProtocolListener;
-import org.apache.nifi.cluster.protocol.message.DisconnectMessage;
-import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
-import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
-import org.apache.nifi.cluster.protocol.message.PrimaryRoleAssignmentMessage;
-import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
-import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage;
-import org.apache.nifi.reporting.BulletinRepository;
-
-/**
- * A wrapper class for consolidating a protocol sender and listener for the cluster
- * manager.
- * 
- * @author unattributed
- */
-public class ClusterManagerProtocolSenderListener implements ClusterManagerProtocolSender, ProtocolListener {
-    
-    private final ClusterManagerProtocolSender sender;
-    
-    private final ProtocolListener listener;
-    
-    public ClusterManagerProtocolSenderListener(final ClusterManagerProtocolSender sender, final ProtocolListener listener) {
-        if(sender == null) {
-            throw new IllegalArgumentException("ClusterManagerProtocolSender may not be null.");
-        } else if(listener == null) {
-            throw new IllegalArgumentException("ProtocolListener may not be null.");
-        }
-        this.sender = sender;
-        this.listener = listener;
-    }
-
-    @Override
-    public void stop() throws IOException {
-        if(!isRunning()) {
-            throw new IllegalStateException("Instance is already stopped.");
-        }
-        listener.stop();
-    }
-
-    @Override
-    public void start() throws IOException {
-        if(isRunning()) {
-            throw new IllegalStateException("Instance is already started.");
-        }
-        listener.start();
-    }
-
-    @Override
-    public boolean isRunning() {
-        return listener.isRunning();
-    }
-
-    @Override
-    public boolean removeHandler(final ProtocolHandler handler) {
-        return listener.removeHandler(handler);
-    }
-
-    @Override
-    public Collection<ProtocolHandler> getHandlers() {
-        return listener.getHandlers();
-    }
-
-    @Override
-    public void addHandler(final ProtocolHandler handler) {
-        listener.addHandler(handler);
-    }
-    
-    @Override
-    public void setBulletinRepository(final BulletinRepository bulletinRepository) {
-        listener.setBulletinRepository(bulletinRepository);
-        sender.setBulletinRepository(bulletinRepository);
-    }
-    
-    @Override
-    public FlowResponseMessage requestFlow(final FlowRequestMessage msg) throws ProtocolException {
-        return sender.requestFlow(msg);
-    }
-
-    @Override
-    public ReconnectionResponseMessage requestReconnection(final ReconnectionRequestMessage msg) throws ProtocolException {
-        return sender.requestReconnection(msg);
-    }
-
-    @Override
-    public void disconnect(DisconnectMessage msg) throws ProtocolException {
-        sender.disconnect(msg);
-    }
-    
-    @Override
-    public void assignPrimaryRole(PrimaryRoleAssignmentMessage msg) throws ProtocolException {
-        sender.assignPrimaryRole(msg);
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java
deleted file mode 100644
index 24e51e0..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java
+++ /dev/null
@@ -1,181 +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.cluster.protocol.impl;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.Collection;
-import java.util.Collections;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.apache.nifi.io.socket.multicast.MulticastServiceDiscovery;
-import org.apache.nifi.reporting.BulletinRepository;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.ProtocolListener;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
-import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An implementation for discovering services by way of "service broadcast" type
- * protocol messages over multicast.
- *
- * The client caller is responsible for starting and stopping the service
- * discovery. The instance must be stopped before termination of the JVM to
- * ensure proper resource clean-up.
- *
- * @author unattributed
- */
-public class ClusterServiceDiscovery implements MulticastServiceDiscovery, ProtocolListener {
-
-    private static final Logger logger = LoggerFactory.getLogger(ClusterServiceDiscovery.class);
-    private final String serviceName;
-    private final MulticastConfiguration multicastConfiguration;
-    private final MulticastProtocolListener listener;
-    private volatile BulletinRepository bulletinRepository;
-
-    /*
-     * guarded by this
-     */
-    private DiscoverableService service;
-
-    
-    public ClusterServiceDiscovery(final String serviceName, final InetSocketAddress multicastAddress,
-            final MulticastConfiguration multicastConfiguration, final ProtocolContext<ProtocolMessage> protocolContext) {
-
-        if (StringUtils.isBlank(serviceName)) {
-            throw new IllegalArgumentException("Service name may not be null or empty.");
-        } else if (multicastAddress == null) {
-            throw new IllegalArgumentException("Multicast address may not be null.");
-        } else if (multicastAddress.getAddress().isMulticastAddress() == false) {
-            throw new IllegalArgumentException("Multicast group must be a Class D address.");
-        } else if (protocolContext == null) {
-            throw new IllegalArgumentException("Protocol Context may not be null.");
-        } else if (multicastConfiguration == null) {
-            throw new IllegalArgumentException("Multicast configuration may not be null.");
-        }
-
-        this.serviceName = serviceName;
-        this.multicastConfiguration = multicastConfiguration;
-        this.listener = new MulticastProtocolListener(1, multicastAddress, multicastConfiguration, protocolContext);
-        listener.addHandler(new ClusterManagerServiceBroadcastHandler());
-    }
-
-    @Override
-    public void setBulletinRepository(final BulletinRepository bulletinRepository) {
-        this.bulletinRepository = bulletinRepository;
-    }
-
-    @Override
-    public synchronized DiscoverableService getService() {
-        return service;
-    }
-
-    @Override
-    public InetSocketAddress getMulticastAddress() {
-        return listener.getMulticastAddress();
-    }
-
-    @Override
-    public Collection<ProtocolHandler> getHandlers() {
-        return Collections.unmodifiableCollection(listener.getHandlers());
-    }
-
-    @Override
-    public void addHandler(ProtocolHandler handler) {
-        listener.addHandler(handler);
-    }
-
-    @Override
-    public boolean removeHandler(ProtocolHandler handler) {
-        return listener.removeHandler(handler);
-    }
-
-    @Override
-    public boolean isRunning() {
-        return listener.isRunning();
-    }
-
-    @Override
-    public void start() throws IOException {
-        if (isRunning()) {
-            throw new IllegalStateException("Instance is already running.");
-        }
-        listener.start();
-    }
-
-    @Override
-    public void stop() throws IOException {
-        if (isRunning() == false) {
-            throw new IllegalStateException("Instance is already stopped.");
-        }
-        listener.stop();
-    }
-
-    public String getServiceName() {
-        return serviceName;
-    }
-
-    public MulticastConfiguration getMulticastConfiguration() {
-        return multicastConfiguration;
-    }
-
-    private class ClusterManagerServiceBroadcastHandler implements ProtocolHandler {
-
-        @Override
-        public boolean canHandle(final ProtocolMessage msg) {
-            return MessageType.SERVICE_BROADCAST == msg.getType();
-        }
-
-        @Override
-        public ProtocolMessage handle(final ProtocolMessage msg) throws ProtocolException {
-            synchronized (ClusterServiceDiscovery.this) {
-                if (canHandle(msg) == false) {
-                    throw new ProtocolException("Handler cannot handle message type: " + msg.getType());
-                } else {
-                    final ServiceBroadcastMessage broadcastMsg = (ServiceBroadcastMessage) msg;
-                    if (serviceName.equals(broadcastMsg.getServiceName())) {
-                        final DiscoverableService oldService = service;
-                        if (oldService == null
-                                || broadcastMsg.getAddress().equalsIgnoreCase(oldService.getServiceAddress().getHostName()) == false
-                                || broadcastMsg.getPort() != oldService.getServiceAddress().getPort()) {
-                            service = new DiscoverableServiceImpl(serviceName, InetSocketAddress.createUnresolved(broadcastMsg.getAddress(), broadcastMsg.getPort()));
-                            final InetSocketAddress oldServiceAddress = (oldService == null) ? null : oldService.getServiceAddress();
-                            logger.info(String.format("Updating cluster service address for '%s' from '%s' to '%s'", serviceName, prettyPrint(oldServiceAddress), prettyPrint(service.getServiceAddress())));
-                        }
-                    }
-                    return null;
-                }
-            }
-        }
-    }
-
-    private String prettyPrint(final InetSocketAddress address) {
-        if (address == null) {
-            return "0.0.0.0:0";
-        } else {
-            return address.getHostName() + ":" + address.getPort();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java
deleted file mode 100644
index bebfde8..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java
+++ /dev/null
@@ -1,229 +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.cluster.protocol.impl;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-import org.apache.nifi.io.socket.multicast.ServiceDiscovery;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Implements the ServiceLocator interface for locating the socket address
- * of a cluster service.  Depending on configuration, the address may be located
- * using service discovery.  If using service discovery, then the service methods
- * must be used for starting and stopping discovery.
- * 
- * Service discovery may be used in conjunction with a fixed port.  In this case,
- * the service discovery will yield the service IP/host while the fixed port will
- * be used for the port.
- * 
- * Alternatively, the instance may be configured with exact service location, in
- * which case, no service discovery occurs and the caller will always receive the
- * configured service.
- * 
- * @author unattributed
- */
-public class ClusterServiceLocator implements ServiceDiscovery {
-    
-    private static final Logger logger = LoggerFactory.getLogger(ClusterServiceLocator.class);
-    
-    private final String serviceName;
-    
-    private final ClusterServiceDiscovery serviceDiscovery;
-    
-    private final DiscoverableService fixedService;
-
-    private final int fixedServicePort;
-    
-    private final AttemptsConfig attemptsConfig = new AttemptsConfig();
-    
-    private final AtomicBoolean running = new AtomicBoolean(false);
-    
-    public ClusterServiceLocator(final ClusterServiceDiscovery serviceDiscovery) {
-        if(serviceDiscovery == null) {
-            throw new IllegalArgumentException("ClusterServiceDiscovery may not be null.");
-        }
-        this.serviceDiscovery = serviceDiscovery;
-        this.fixedService = null;
-        this.fixedServicePort = 0;
-        this.serviceName = serviceDiscovery.getServiceName();
-    }
-    
-    public ClusterServiceLocator(final ClusterServiceDiscovery serviceDiscovery, final int fixedServicePort) {
-        if(serviceDiscovery == null) {
-            throw new IllegalArgumentException("ClusterServiceDiscovery may not be null.");
-        }
-        this.serviceDiscovery = serviceDiscovery;
-        this.fixedService = null;
-        this.fixedServicePort = fixedServicePort;
-        this.serviceName = serviceDiscovery.getServiceName();
-    }
-    
-    public ClusterServiceLocator(final DiscoverableService fixedService) {
-        if(fixedService == null) {
-            throw new IllegalArgumentException("Service may not be null.");
-        }
-        this.serviceDiscovery = null;
-        this.fixedService = fixedService;
-        this.fixedServicePort = 0;
-        this.serviceName = fixedService.getServiceName();
-    }
-    
-    @Override
-    public DiscoverableService getService() {
-        
-        final int numAttemptsValue;
-        final int secondsBetweenAttempts;
-        synchronized(this) {
-            numAttemptsValue = attemptsConfig.numAttempts;
-            secondsBetweenAttempts = attemptsConfig.getTimeBetweenAttempts();
-        }
-        
-        // try for a configured amount of attempts to retrieve the service address
-        for(int i = 0; i < numAttemptsValue; i++) {
-
-            if(fixedService != null) {
-                return fixedService;
-            } else if(serviceDiscovery != null) {
-                
-                final DiscoverableService discoveredService = serviceDiscovery.getService();
-                
-                // if we received an address
-                if(discoveredService != null) {
-                    // if we were configured with a fixed port, then use the discovered host and fixed port; otherwise use the discovered address
-                    if(fixedServicePort > 0) {
-                        // create service using discovered service name and address with fixed service port
-                        final InetSocketAddress addr = InetSocketAddress.createUnresolved(discoveredService.getServiceAddress().getHostName(), fixedServicePort);
-                        final DiscoverableService result = new DiscoverableServiceImpl(discoveredService.getServiceName(), addr);
-                        return result;
-                    } else {
-                        return discoveredService;
-                    }
-                }
-            }
-            
-            // could not obtain service address, so sleep a bit
-            try {
-                logger.debug(String.format("Locating Cluster Service '%s' Attempt: %d of %d failed.  Trying again in %d seconds.", 
-                    serviceName, (i + 1), numAttemptsValue, secondsBetweenAttempts));
-                Thread.sleep(secondsBetweenAttempts * 1000);
-            } catch(final InterruptedException ie) {
-                break;
-            }
-            
-        }
-
-        return null;
-    }
-
-    public boolean isRunning() {
-        if(serviceDiscovery != null) {
-            return serviceDiscovery.isRunning();
-        } else {
-            return running.get();
-        }
-    }
-
-    public void start() throws IOException {
-        
-        if(isRunning()) {
-            throw new IllegalStateException("Instance is already started.");
-        }
-        
-        if(serviceDiscovery != null) {
-            serviceDiscovery.start();
-        }
-        running.set(true);
-    }
-
-    public void stop() throws IOException {
-        
-        if(isRunning() == false) {
-            throw new IllegalStateException("Instance is already stopped.");
-        }
-        
-        if(serviceDiscovery != null) {
-            serviceDiscovery.stop();
-        }
-        running.set(false);
-    }
-    
-    public synchronized void setAttemptsConfig(final AttemptsConfig config) {
-        if(config == null) {
-            throw new IllegalArgumentException("Attempts configuration may not be null.");
-        }
-        this.attemptsConfig.numAttempts = config.numAttempts;
-        this.attemptsConfig.timeBetweenAttempts = config.timeBetweenAttempts;
-        this.attemptsConfig.timeBetweenAttempsUnit = config.timeBetweenAttempsUnit;
-    }
-
-    public synchronized AttemptsConfig getAttemptsConfig() {
-        final AttemptsConfig config = new AttemptsConfig();
-        config.numAttempts = this.attemptsConfig.numAttempts;
-        config.timeBetweenAttempts = this.attemptsConfig.timeBetweenAttempts;
-        config.timeBetweenAttempsUnit = this.attemptsConfig.timeBetweenAttempsUnit;
-        return config;
-    }
-    
-    public static class AttemptsConfig {
-        
-        private int numAttempts = 1;
-        
-        private int timeBetweenAttempts = 1;
-        
-        private TimeUnit timeBetweenAttempsUnit = TimeUnit.SECONDS;
-        
-        public int getNumAttempts() {
-            return numAttempts;
-        }
-
-        public void setNumAttempts(int numAttempts) {
-            if(numAttempts <= 0) {
-                throw new IllegalArgumentException("Number of attempts must be positive: " + numAttempts);
-            }
-            this.numAttempts = numAttempts;
-        }
-
-        public TimeUnit getTimeBetweenAttemptsUnit() {
-            return timeBetweenAttempsUnit;
-        }
-
-        public void setTimeBetweenAttempsUnit(TimeUnit timeBetweenAttempsUnit) {
-            if(timeBetweenAttempts <= 0) {
-                throw new IllegalArgumentException("Time between attempts must be positive: " + numAttempts);
-            } 
-            this.timeBetweenAttempsUnit = timeBetweenAttempsUnit;
-        }
-
-        public int getTimeBetweenAttempts() {
-            return timeBetweenAttempts;
-        }
-
-        public void setTimeBetweenAttempts(int timeBetweenAttempts) {
-            if(timeBetweenAttempts <= 0) {
-            throw new IllegalArgumentException("Time between attempts must be positive: " + numAttempts);
-        } 
-            this.timeBetweenAttempts = timeBetweenAttempts;
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java
deleted file mode 100644
index e9e7d5b..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java
+++ /dev/null
@@ -1,182 +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.cluster.protocol.impl;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.net.DatagramPacket;
-import java.net.InetSocketAddress;
-import java.net.MulticastSocket;
-import java.util.*;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.apache.nifi.io.socket.multicast.MulticastServicesBroadcaster;
-import org.apache.nifi.io.socket.multicast.MulticastUtils;
-import org.apache.nifi.logging.NiFiLog;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Broadcasts services used by the clustering software using multicast communication.
- * A configurable delay occurs after broadcasting the collection of services.
- * 
- * The client caller is responsible for starting and stopping the broadcasting.
- * The instance must be stopped before termination of the JVM to ensure proper
- * resource clean-up.
- * 
- * @author unattributed
- */
-public class ClusterServicesBroadcaster implements MulticastServicesBroadcaster {
-    
-    private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(ClusterServicesBroadcaster.class));
-    
-    private final Set<DiscoverableService> services = new CopyOnWriteArraySet<>();
-
-    private final InetSocketAddress multicastAddress;
-    
-    private final MulticastConfiguration multicastConfiguration;
-    
-    private final ProtocolContext<ProtocolMessage> protocolContext;
-    
-    private final int broadcastDelayMs;
-    
-    private Timer broadcaster;
-    
-    private MulticastSocket multicastSocket;
-    
-    public ClusterServicesBroadcaster(final InetSocketAddress multicastAddress, 
-            final MulticastConfiguration multicastConfiguration, 
-            final ProtocolContext<ProtocolMessage> protocolContext, final String broadcastDelay) {
-        
-        if(multicastAddress == null) {
-            throw new IllegalArgumentException("Multicast address may not be null.");
-        } else if(multicastAddress.getAddress().isMulticastAddress() == false) {
-            throw new IllegalArgumentException("Multicast group address is not a Class D IP address.");
-        } else if(protocolContext == null) {
-            throw new IllegalArgumentException("Protocol Context may not be null.");
-        } else if(multicastConfiguration == null) {
-            throw new IllegalArgumentException("Multicast configuration may not be null.");
-        }
-        
-        this.services.addAll(services);
-        this.multicastAddress = multicastAddress;
-        this.multicastConfiguration = multicastConfiguration;
-        this.protocolContext = protocolContext;
-        this.broadcastDelayMs = (int) FormatUtils.getTimeDuration(broadcastDelay, TimeUnit.MILLISECONDS);
-    }
-    
-    public void start() throws IOException {
-
-        if(isRunning()) {
-            throw new IllegalStateException("Instance is already started.");
-        }
-        
-        // setup socket
-        multicastSocket = MulticastUtils.createMulticastSocket(multicastConfiguration);
-        
-        // setup broadcaster
-        broadcaster = new Timer("Cluster Services Broadcaster", /* is daemon */ true);
-        broadcaster.schedule(new TimerTask() {
-            @Override
-            public void run() {
-                for(final DiscoverableService service : services) {
-                    try {
-
-                        final InetSocketAddress serviceAddress = service.getServiceAddress();
-                        logger.debug(String.format("Broadcasting Cluster Service '%s' at address %s:%d", 
-                            service.getServiceName(), serviceAddress.getHostName(), serviceAddress.getPort()));
-                        
-                        // create message
-                        final ServiceBroadcastMessage msg = new ServiceBroadcastMessage();
-                        msg.setServiceName(service.getServiceName());
-                        msg.setAddress(serviceAddress.getHostName());
-                        msg.setPort(serviceAddress.getPort());
-
-                        // marshal message to output stream
-                        final ProtocolMessageMarshaller<ProtocolMessage> marshaller = protocolContext.createMarshaller();
-                        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-                        marshaller.marshal(msg, baos);
-                        final byte[] packetBytes = baos.toByteArray();
-
-                        // send message
-                        final DatagramPacket packet = new DatagramPacket(packetBytes, packetBytes.length, multicastAddress);
-                        multicastSocket.send(packet);
-
-                    } catch(final Exception ex) {
-                        logger.warn(String.format("Cluster Services Broadcaster failed broadcasting service '%s' due to: %s", service.getServiceName(), ex), ex);
-                    }
-                }
-            }
-        }, 0, broadcastDelayMs);
-    }
-    
-    public boolean isRunning() {
-        return (broadcaster != null);
-    }
-    
-    public void stop() {
-        
-        if(isRunning() == false) {
-            throw new IllegalStateException("Instance is already stopped.");
-        }
-        
-        broadcaster.cancel();
-        broadcaster = null;
-
-        // close socket
-        MulticastUtils.closeQuietly(multicastSocket);
-        
-    }
-
-    @Override
-    public int getBroadcastDelayMs() {
-        return broadcastDelayMs;
-    }
-    
-    @Override
-    public Set<DiscoverableService> getServices() {
-        return Collections.unmodifiableSet(services);
-    }
-
-    @Override
-    public InetSocketAddress getMulticastAddress() {
-        return multicastAddress;
-    }
-    
-    @Override
-    public boolean addService(final DiscoverableService service) {
-        return services.add(service);
-    }
-    
-    @Override
-    public boolean removeService(final String serviceName) {
-        for(final DiscoverableService service : services) {
-            if(service.getServiceName().equals(serviceName)) {
-                return services.remove(service);
-            }
-        }
-        return false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/CopyingInputStream.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/CopyingInputStream.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/CopyingInputStream.java
deleted file mode 100644
index 680df65..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/CopyingInputStream.java
+++ /dev/null
@@ -1,77 +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.cluster.protocol.impl;
-
-import java.io.ByteArrayOutputStream;
-import java.io.FilterInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-public class CopyingInputStream extends FilterInputStream {
-    private final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    private final int maxBytesToCopy;
-    private final InputStream in;
-
-    public CopyingInputStream(final InputStream in, final int maxBytesToCopy) {
-        super(in);
-        this.maxBytesToCopy = maxBytesToCopy;
-        this.in = in;
-    }
-    
-    @Override
-    public int read() throws IOException {
-        final int delegateRead = in.read();
-        if ( delegateRead != -1 && getNumberOfBytesCopied() < maxBytesToCopy ) {
-            baos.write(delegateRead);
-        }
-        
-        return delegateRead;
-    }
-    
-    @Override
-    public int read(byte[] b) throws IOException {
-        final int delegateRead = in.read(b);
-        if ( delegateRead >= 0 ) {
-            baos.write(b, 0, Math.min(delegateRead, maxBytesToCopy - getNumberOfBytesCopied()));
-        }
-        
-        return delegateRead;
-    }
-    
-    @Override
-    public int read(byte[] b, int off, int len) throws IOException {
-        final int delegateRead = in.read(b, off, len);
-        if ( delegateRead >= 0 ) {
-            baos.write(b, off, Math.min(delegateRead, maxBytesToCopy - getNumberOfBytesCopied()));
-        }
-        
-        return delegateRead;
-    }
-    
-    public byte[] getBytesRead() {
-        return baos.toByteArray();
-    }
-    
-    public void writeBytes(final OutputStream out) throws IOException {
-        baos.writeTo(out);
-    }
-    
-    public int getNumberOfBytesCopied() {
-        return baos.size();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java
deleted file mode 100644
index d3764b3..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java
+++ /dev/null
@@ -1,204 +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.cluster.protocol.impl;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.net.DatagramPacket;
-import java.net.InetSocketAddress;
-import java.net.MulticastSocket;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.UUID;
-import java.util.concurrent.CopyOnWriteArrayList;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.apache.nifi.io.socket.multicast.MulticastListener;
-import org.apache.nifi.logging.NiFiLog;
-import org.apache.nifi.reporting.Bulletin;
-import org.apache.nifi.reporting.BulletinRepository;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.ProtocolListener;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller;
-import org.apache.nifi.cluster.protocol.message.MulticastProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.events.BulletinFactory;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Implements a listener for protocol messages sent over multicast.  If a message
- * is of type MulticastProtocolMessage, then the underlying protocol message is
- * passed to the handler.  If the receiving handler produces a message response,
- * then the message is wrapped with a MulticastProtocolMessage before being 
- * sent to the originator.
- * 
- * The client caller is responsible for starting and stopping the listener.
- * The instance must be stopped before termination of the JVM to ensure proper
- * resource clean-up.
- * 
- * @author unattributed
- */
-public class MulticastProtocolListener extends MulticastListener implements ProtocolListener {
-    
-    private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(MulticastProtocolListener.class));
-
-    // immutable members
-    private final Collection<ProtocolHandler> handlers = new CopyOnWriteArrayList<>();
-    private final String listenerId = UUID.randomUUID().toString();
-    private final ProtocolContext<ProtocolMessage> protocolContext;
-    private volatile BulletinRepository bulletinRepository;
-
-    public MulticastProtocolListener(
-            final int numThreads,
-            final InetSocketAddress multicastAddress,
-            final MulticastConfiguration configuration,
-            final ProtocolContext<ProtocolMessage> protocolContext) {
-
-        super(numThreads, multicastAddress, configuration);
-        
-        if (protocolContext == null) {
-            throw new IllegalArgumentException("Protocol Context may not be null.");
-        }
-        this.protocolContext = protocolContext;
-    }
-
-    @Override
-    public void setBulletinRepository(final BulletinRepository bulletinRepository) {
-        this.bulletinRepository = bulletinRepository;
-    }
-
-    @Override
-    public void start() throws IOException {
-
-        if(super.isRunning()) {
-            throw new IllegalStateException("Instance is already started.");
-        }
-        
-        super.start();
-        
-    }
-
-    @Override
-    public void stop() throws IOException {
-
-        if(super.isRunning() == false) {
-            throw new IllegalStateException("Instance is already stopped.");
-        }
-        
-        // shutdown listener
-        super.stop();
-
-    }
-
-    @Override
-    public Collection<ProtocolHandler> getHandlers() {
-        return Collections.unmodifiableCollection(handlers);
-    }
-
-    @Override
-    public void addHandler(final ProtocolHandler handler) {
-        if(handler == null) {
-            throw new NullPointerException("Protocol handler may not be null.");
-        }
-        handlers.add(handler);
-    }
-    
-    @Override
-    public boolean removeHandler(final ProtocolHandler handler) {
-        return handlers.remove(handler);
-    }
-    
-    @Override
-    public void dispatchRequest(final MulticastSocket multicastSocket, final DatagramPacket packet) {
-
-        try {
-
-            // unmarshall message
-            final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = protocolContext.createUnmarshaller();
-            final ProtocolMessage request = unmarshaller.unmarshal(new ByteArrayInputStream(packet.getData(), 0, packet.getLength()));
-
-            // unwrap multicast message, if necessary
-            final ProtocolMessage unwrappedRequest;
-            if(request instanceof MulticastProtocolMessage) {
-                final MulticastProtocolMessage multicastRequest = (MulticastProtocolMessage) request;
-                // don't process a message we sent
-                if(listenerId.equals(multicastRequest.getId())) {
-                    return;
-                } else {
-                    unwrappedRequest = multicastRequest.getProtocolMessage();
-                }
-            } else {
-                unwrappedRequest = request;
-            }
-            
-            // dispatch message to handler
-            ProtocolHandler desiredHandler = null;
-            for (final ProtocolHandler handler : getHandlers()) {
-                if (handler.canHandle(unwrappedRequest)) {
-                    desiredHandler = handler;
-                    break;
-                }
-            }
-
-            // if no handler found, throw exception; otherwise handle request
-            if (desiredHandler == null) {
-                throw new ProtocolException("No handler assigned to handle message type: " + request.getType());
-            } else {
-                final ProtocolMessage response = desiredHandler.handle(request);
-                if(response != null) {
-                    try {
-                        
-                        // wrap with listener id
-                        final MulticastProtocolMessage multicastResponse = new MulticastProtocolMessage(listenerId, response);
-                        
-                        // marshal message
-                        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-                        final ProtocolMessageMarshaller<ProtocolMessage> marshaller = protocolContext.createMarshaller();
-                        marshaller.marshal(multicastResponse, baos);
-                        final byte[] responseBytes = baos.toByteArray();
-                        
-                        final int maxPacketSizeBytes = getMaxPacketSizeBytes();
-                        if(responseBytes.length > maxPacketSizeBytes) {
-                            logger.warn("Cluster protocol handler '" + desiredHandler.getClass() + 
-                                "' produced a multicast response with length greater than configured max packet size '" + maxPacketSizeBytes + "'");
-                        }
-                        
-                        // create and send packet
-                        final DatagramPacket responseDatagram = new DatagramPacket(responseBytes, responseBytes.length, getMulticastAddress().getAddress(), getMulticastAddress().getPort()); 
-                        multicastSocket.send(responseDatagram);
-                        
-                    } catch (final IOException ioe) {
-                        throw new ProtocolException("Failed marshalling protocol message in response to message type: " + request.getType() + " due to: " + ioe, ioe);
-                    }
-                }
-            }
-
-        } catch (final Throwable t) {
-            logger.warn("Failed processing protocol message due to " + t, t);
-            
-            if ( bulletinRepository != null ) {
-                final Bulletin bulletin = BulletinFactory.createBulletin("Clustering", "WARNING", "Failed to process Protocol Message due to " + t.toString());
-                bulletinRepository.addBulletin(bulletin);
-            }
-        }
-    }
-}


[41/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/ChannelDispatcher.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/ChannelDispatcher.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/ChannelDispatcher.java
deleted file mode 100644
index 2ae2c07..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/ChannelDispatcher.java
+++ /dev/null
@@ -1,160 +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.io.nio;
-
-import java.io.IOException;
-import java.nio.channels.DatagramChannel;
-import java.nio.channels.SelectableChannel;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.Selector;
-import java.nio.channels.ServerSocketChannel;
-import java.nio.channels.SocketChannel;
-import java.util.Iterator;
-import java.util.UUID;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.nifi.io.nio.consumer.StreamConsumerFactory;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- *
- * @author none
- */
-public final class ChannelDispatcher implements Runnable {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(ChannelDispatcher.class);
-    private final Selector serverSocketSelector;
-    private final Selector socketChannelSelector;
-    private final ScheduledExecutorService executor;
-    private final BufferPool emptyBuffers;
-    private final StreamConsumerFactory factory;
-    private final AtomicLong channelReaderFrequencyMilliseconds = new AtomicLong(DEFAULT_CHANNEL_READER_PERIOD_MILLISECONDS);
-    private final long timeout;
-    private volatile boolean stop = false;
-    public static final long DEFAULT_CHANNEL_READER_PERIOD_MILLISECONDS = 100L;
-
-    public ChannelDispatcher(final Selector serverSocketSelector, final Selector socketChannelSelector, final ScheduledExecutorService service,
-            final StreamConsumerFactory factory, final BufferPool buffers, final long timeout, final TimeUnit unit) {
-        this.serverSocketSelector = serverSocketSelector;
-        this.socketChannelSelector = socketChannelSelector;
-        this.executor = service;
-        this.factory = factory;
-        emptyBuffers = buffers;
-        this.timeout = TimeUnit.MILLISECONDS.convert(timeout, unit);
-    }
-
-    public void setChannelReaderFrequency(final long period, final TimeUnit timeUnit) {
-        channelReaderFrequencyMilliseconds.set(TimeUnit.MILLISECONDS.convert(period, timeUnit));
-    }
-
-    @Override
-    public void run() {
-        while (!stop) {
-            try {
-                selectServerSocketKeys();
-                selectSocketChannelKeys();
-            } catch (final Exception ex) {
-                LOGGER.warn("Key selection failed: {} Normal during shutdown.", new Object[]{ex});
-            }
-        }
-    }
-
-    /*
-     * When serverSocketsChannels are registered with the selector, want each invoke of this method to loop through all
-     * channels' keys.
-     * 
-     * @throws IOException
-     */
-    private void selectServerSocketKeys() throws IOException {
-        int numSelected = serverSocketSelector.select(timeout);
-        if (numSelected == 0) {
-            return;
-        }
-
-        // for each registered server socket - see if any connections are waiting to be established
-        final Iterator<SelectionKey> itr = serverSocketSelector.selectedKeys().iterator();
-        while (itr.hasNext()) {
-            SelectionKey serverSocketkey = itr.next();
-            final SelectableChannel channel = serverSocketkey.channel();
-            AbstractChannelReader reader = null;
-            if (serverSocketkey.isValid() && serverSocketkey.isAcceptable()) {
-                final ServerSocketChannel ssChannel = (ServerSocketChannel) serverSocketkey.channel();
-                final SocketChannel sChannel = ssChannel.accept();
-                if (sChannel != null) {
-                    sChannel.configureBlocking(false);
-                    final SelectionKey socketChannelKey = sChannel.register(socketChannelSelector, SelectionKey.OP_READ);
-                    final String readerId = sChannel.socket().toString();
-                    reader = new SocketChannelReader(readerId, socketChannelKey, emptyBuffers, factory);
-                    final ScheduledFuture<?> readerFuture = executor.scheduleWithFixedDelay(reader, 10L,
-                            channelReaderFrequencyMilliseconds.get(), TimeUnit.MILLISECONDS);
-                    reader.setScheduledFuture(readerFuture);
-                    socketChannelKey.attach(reader);
-                }
-            }
-            itr.remove(); // do this so that the next select operation returns a positive value; otherwise, it will return 0.
-            if (reader != null && LOGGER.isDebugEnabled()) {
-                LOGGER.debug(this + " New Connection established.  Server channel: " + channel + " Reader: " + reader);
-            }
-        }
-    }
-
-    /*
-     * When invoking this method, only want to iterate through the selected keys once. When a key is entered into the selectors
-     * selected key set, select will return a positive value. The next select will return 0 if nothing has changed. Note that
-     * the selected key set is not manually changed via a remove operation.
-     * 
-     * @throws IOException
-     */
-    private void selectSocketChannelKeys() throws IOException {
-        // once a channel associated with a key in this selector is 'ready', it causes this select to immediately return.
-        // thus, for each trip through the run() we only get hit with one real timeout...the one in selectServerSocketKeys.
-        int numSelected = socketChannelSelector.select(timeout);
-        if (numSelected == 0) {
-            return;
-        }
-
-        for (SelectionKey socketChannelKey : socketChannelSelector.selectedKeys()) {
-            final SelectableChannel channel = socketChannelKey.channel();
-            AbstractChannelReader reader = null;
-            // there are 2 kinds of channels in this selector, both which have their own readers and are executed in their own
-            // threads. We will get here whenever a new SocketChannel is created due to an incoming connection. However,
-            // for a DatagramChannel we don't want to create a new reader unless it is a new DatagramChannel. The only
-            // way to tell if it's new is the lack of an attachment. 
-            if (channel instanceof DatagramChannel && socketChannelKey.attachment() == null) {
-                reader = new DatagramChannelReader(UUID.randomUUID().toString(), socketChannelKey, emptyBuffers, factory);
-                socketChannelKey.attach(reader);
-                final ScheduledFuture<?> readerFuture = executor.scheduleWithFixedDelay(reader, 10L, channelReaderFrequencyMilliseconds.get(),
-                        TimeUnit.MILLISECONDS);
-                reader.setScheduledFuture(readerFuture);
-            }
-            if (reader != null && LOGGER.isDebugEnabled()) {
-                LOGGER.debug(this + " New Connection established.  Server channel: " + channel + " Reader: " + reader);
-            }
-        }
-
-    }
-
-    public void stop() {
-        stop = true;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/ChannelListener.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/ChannelListener.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/ChannelListener.java
deleted file mode 100644
index b0a1cfb..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/ChannelListener.java
+++ /dev/null
@@ -1,228 +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.io.nio;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.StandardSocketOptions;
-import java.nio.channels.DatagramChannel;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.Selector;
-import java.nio.channels.ServerSocketChannel;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.io.nio.consumer.StreamConsumerFactory;
-
-import org.apache.commons.io.IOUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class provides the entry point to NIO based socket listeners for NiFi
- * processors and services. There are 2 supported types of Listeners, Datagram
- * (UDP based transmissions) and ServerSocket (TCP based transmissions). This
- * will create the ChannelDispatcher, which is a Runnable and is controlled via
- * the ScheduledExecutorService, which is also created by this class. The
- * ChannelDispatcher handles connections to the ServerSocketChannels and creates
- * the readers associated with the resulting SocketChannels. Additionally, this
- * creates and manages two Selectors, one for ServerSocketChannels and another
- * for SocketChannels and DatagramChannels.
- *
- * The threading model for this consists of one thread for the
- * ChannelDispatcher, one thread per added SocketChannel reader, one thread per
- * added DatagramChannel reader. The ChannelDispatcher is not scheduled with
- * fixed delay as the others are. It is throttled by the provided timeout value.
- * Within the ChannelDispatcher there are two blocking operations which will
- * block for the given timeout each time through the enclosing loop.
- *
- * All channels are cached in one of the two Selectors via their SelectionKey.
- * The serverSocketSelector maintains all the added ServerSocketChannels; the
- * socketChannelSelector maintains the all the add DatagramChannels and the
- * created SocketChannels. Further, the SelectionKey of the DatagramChannel and
- * the SocketChannel is injected with the channel's associated reader.
- *
- * All ChannelReaders will get throttled by the unavailability of buffers in the
- * provided BufferPool. This is designed to create back pressure.
- *
- * @author none
- */
-public final class ChannelListener {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(ChannelListener.class);
-    private final ScheduledExecutorService executor;
-    private final Selector serverSocketSelector; // used to listen for new connections
-    private final Selector socketChannelSelector; // used to listen on existing connections
-    private final ChannelDispatcher channelDispatcher;
-    private final BufferPool bufferPool;
-    private final int initialBufferPoolSize;
-    private volatile long channelReaderFrequencyMSecs = 50;
-
-    public ChannelListener(final int threadPoolSize, final StreamConsumerFactory consumerFactory, final BufferPool bufferPool, int timeout,
-            TimeUnit unit) throws IOException {
-        this.executor = Executors.newScheduledThreadPool(threadPoolSize + 1); // need to allow for long running ChannelDispatcher thread
-        this.serverSocketSelector = Selector.open();
-        this.socketChannelSelector = Selector.open();
-        this.bufferPool = bufferPool;
-        this.initialBufferPoolSize = bufferPool.size();
-        channelDispatcher = new ChannelDispatcher(serverSocketSelector, socketChannelSelector, executor, consumerFactory, bufferPool,
-                timeout, unit);
-        executor.schedule(channelDispatcher, 50, TimeUnit.MILLISECONDS);
-    }
-
-    public void setChannelReaderSchedulingPeriod(final long period, final TimeUnit unit) {
-        channelReaderFrequencyMSecs = TimeUnit.MILLISECONDS.convert(period, unit);
-        channelDispatcher.setChannelReaderFrequency(period, unit);
-    }
-
-    /**
-     * Adds a server socket channel for listening to connections.
-     *
-     * @param nicIPAddress - if null binds to wildcard address
-     * @param port - port to bind to
-     * @param receiveBufferSize - size of OS receive buffer to request. If less
-     * than 0 then will not be set and OS default will win.
-     * @throws IOException
-     */
-    public void addServerSocket(final InetAddress nicIPAddress, final int port, final int receiveBufferSize)
-            throws IOException {
-        final ServerSocketChannel ssChannel = ServerSocketChannel.open();
-        ssChannel.configureBlocking(false);
-        if (receiveBufferSize > 0) {
-            ssChannel.setOption(StandardSocketOptions.SO_RCVBUF, receiveBufferSize);
-            final int actualReceiveBufSize = ssChannel.getOption(StandardSocketOptions.SO_RCVBUF);
-            if (actualReceiveBufSize < receiveBufferSize) {
-                LOGGER.warn(this + " attempted to set TCP Receive Buffer Size to "
-                        + receiveBufferSize + " bytes but could only set to " + actualReceiveBufSize
-                        + "bytes. You may want to consider changing the Operating System's "
-                        + "maximum receive buffer");
-            }
-        }
-        ssChannel.setOption(StandardSocketOptions.SO_REUSEADDR, true);
-        ssChannel.bind(new InetSocketAddress(nicIPAddress, port));
-        ssChannel.register(serverSocketSelector, SelectionKey.OP_ACCEPT);
-    }
-
-    /**
-     * Binds to listen for data grams on the given local IPAddress/port
-     *
-     * @param nicIPAddress - if null will listen on wildcard address, which
-     * means datagrams will be received on all local network interfaces.
-     * Otherwise, will bind to the provided IP address associated with some NIC.
-     * @param port - the port to listen on
-     * @param receiveBufferSize - the number of bytes to request for a receive
-     * buffer from OS
-     * @throws IOException
-     */
-    public void addDatagramChannel(final InetAddress nicIPAddress, final int port, final int receiveBufferSize)
-            throws IOException {
-        final DatagramChannel dChannel = createAndBindDatagramChannel(nicIPAddress, port, receiveBufferSize);
-        dChannel.register(socketChannelSelector, SelectionKey.OP_READ);
-    }
-
-    /**
-     * Binds to listen for data grams on the given local IPAddress/port and
-     * restricts receipt of datagrams to those from the provided host and port,
-     * must specify both. This improves performance for datagrams coming from a
-     * sender that is known a-priori.
-     *
-     * @param nicIPAddress - if null will listen on wildcard address, which
-     * means datagrams will be received on all local network interfaces.
-     * Otherwise, will bind to the provided IP address associated with some NIC.
-     * @param port - the port to listen on. This is used to provide a well-known
-     * destination for a sender.
-     * @param receiveBufferSize - the number of bytes to request for a receive
-     * buffer from OS
-     * @param sendingHost - the hostname, or IP address, of the sender of
-     * datagrams. Only datagrams from this host will be received. If this is
-     * null the wildcard ip is used, which means datagrams may be received from
-     * any network interface on the local host.
-     * @param sendingPort - the port used by the sender of datagrams. Only
-     * datagrams from this port will be received.
-     * @throws IOException
-     */
-    public void addDatagramChannel(final InetAddress nicIPAddress, final int port, final int receiveBufferSize, final String sendingHost,
-            final Integer sendingPort) throws IOException {
-
-        if (sendingHost == null || sendingPort == null) {
-            addDatagramChannel(nicIPAddress, port, receiveBufferSize);
-            return;
-        }
-        final DatagramChannel dChannel = createAndBindDatagramChannel(nicIPAddress, port, receiveBufferSize);
-        dChannel.connect(new InetSocketAddress(sendingHost, sendingPort));
-        dChannel.register(socketChannelSelector, SelectionKey.OP_READ);
-    }
-
-    private DatagramChannel createAndBindDatagramChannel(final InetAddress nicIPAddress, final int port, final int receiveBufferSize)
-            throws IOException {
-        final DatagramChannel dChannel = DatagramChannel.open();
-        dChannel.configureBlocking(false);
-        if (receiveBufferSize > 0) {
-            dChannel.setOption(StandardSocketOptions.SO_RCVBUF, receiveBufferSize);
-            final int actualReceiveBufSize = dChannel.getOption(StandardSocketOptions.SO_RCVBUF);
-            if (actualReceiveBufSize < receiveBufferSize) {
-                LOGGER.warn(this + " attempted to set UDP Receive Buffer Size to "
-                        + receiveBufferSize + " bytes but could only set to " + actualReceiveBufSize
-                        + "bytes. You may want to consider changing the Operating System's "
-                        + "maximum receive buffer");
-            }
-        }
-        dChannel.setOption(StandardSocketOptions.SO_REUSEADDR, true);
-        dChannel.bind(new InetSocketAddress(nicIPAddress, port));
-        return dChannel;
-    }
-
-    public void shutdown(final long period, final TimeUnit timeUnit) {
-        channelDispatcher.stop();
-        for (SelectionKey selectionKey : socketChannelSelector.keys()) {
-            final AbstractChannelReader reader = (AbstractChannelReader) selectionKey.attachment();
-            selectionKey.cancel();
-            if (reader != null) {
-                while (!reader.isClosed()) {
-                    try {
-                        Thread.sleep(channelReaderFrequencyMSecs);
-                    } catch (InterruptedException e) {
-                    }
-                }
-                final ScheduledFuture<?> readerFuture = reader.getScheduledFuture();
-                readerFuture.cancel(false);
-            }
-            IOUtils.closeQuietly(selectionKey.channel()); // should already be closed via reader, but if reader did not exist...
-        }
-        IOUtils.closeQuietly(socketChannelSelector);
-
-        for (SelectionKey selectionKey : serverSocketSelector.keys()) {
-            selectionKey.cancel();
-            IOUtils.closeQuietly(selectionKey.channel());
-        }
-        IOUtils.closeQuietly(serverSocketSelector);
-        executor.shutdown();
-        try {
-            executor.awaitTermination(period, timeUnit);
-        } catch (final InterruptedException ex) {
-            LOGGER.warn("Interrupted while trying to shutdown executor");
-        }
-        final int currentBufferPoolSize = bufferPool.size();
-        final String warning = (currentBufferPoolSize != initialBufferPoolSize) ? "Initial buffer count=" + initialBufferPoolSize
-                + " Current buffer count=" + currentBufferPoolSize
-                + " Could indicate a buffer leak.  Ensure all consumers are executed until they complete." : "";
-        LOGGER.info("Channel listener shutdown. " + warning);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/DatagramChannelReader.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/DatagramChannelReader.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/DatagramChannelReader.java
deleted file mode 100644
index 1eb5c7e..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/DatagramChannelReader.java
+++ /dev/null
@@ -1,59 +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.io.nio;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.DatagramChannel;
-import java.nio.channels.SelectionKey;
-
-import org.apache.nifi.io.nio.consumer.StreamConsumerFactory;
-
-/**
- *
- * @author none
- */
-public final class DatagramChannelReader extends AbstractChannelReader {
-
-    public static final int MAX_UDP_PACKET_SIZE = 65507;
-
-    public DatagramChannelReader(final String id, final SelectionKey key, final BufferPool empties, final StreamConsumerFactory consumerFactory) {
-        super(id, key, empties, consumerFactory);
-    }
-
-    /**
-     * Will receive UDP data from channel and won't receive anything unless the
-     * given buffer has enough space for at least one full max udp packet.
-     *
-     * @param key
-     * @param buffer
-     * @return
-     * @throws IOException
-     */
-    @Override
-    protected int fillBuffer(final SelectionKey key, final ByteBuffer buffer) throws IOException {
-        final DatagramChannel dChannel = (DatagramChannel) key.channel();
-        final int initialBufferPosition = buffer.position();
-        while (buffer.remaining() > MAX_UDP_PACKET_SIZE && key.isValid() && key.isReadable()) {
-            if (dChannel.receive(buffer) == null) {
-                break;
-            }
-        }
-        return buffer.position() - initialBufferPosition;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/SocketChannelReader.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/SocketChannelReader.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/SocketChannelReader.java
deleted file mode 100644
index db2c102..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/SocketChannelReader.java
+++ /dev/null
@@ -1,55 +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.io.nio;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.SocketChannel;
-import org.apache.nifi.io.nio.consumer.StreamConsumerFactory;
-
-/**
- *
- * @author none
- */
-public final class SocketChannelReader extends AbstractChannelReader {
-
-    public SocketChannelReader(final String id, final SelectionKey key, final BufferPool empties, final StreamConsumerFactory consumerFactory) {
-        super(id, key, empties, consumerFactory);
-    }
-
-    /**
-     * Receives TCP data from the socket channel for the given key.
-     *
-     * @param key
-     * @param buffer
-     * @return
-     * @throws IOException
-     */
-    @Override
-    protected int fillBuffer(final SelectionKey key, final ByteBuffer buffer) throws IOException {
-        int bytesRead = 0;
-        final SocketChannel sChannel = (SocketChannel) key.channel();
-        while (key.isValid() && key.isReadable()) {
-            bytesRead = sChannel.read(buffer);
-            if (bytesRead <= 0) {
-                break;
-            }
-        }
-        return bytesRead;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/AbstractStreamConsumer.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/AbstractStreamConsumer.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/AbstractStreamConsumer.java
deleted file mode 100644
index fce59c6..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/AbstractStreamConsumer.java
+++ /dev/null
@@ -1,132 +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.io.nio.consumer;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.nifi.io.nio.BufferPool;
-import org.apache.commons.lang3.builder.EqualsBuilder;
-import org.apache.commons.lang3.builder.HashCodeBuilder;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-/**
- *
- * @author none
- */
-public abstract class AbstractStreamConsumer implements StreamConsumer {
-
-    private final String uniqueId;
-    private BufferPool bufferPool = null;
-    private final BlockingQueue<ByteBuffer> filledBuffers = new LinkedBlockingQueue<>();
-    private final AtomicBoolean streamEnded = new AtomicBoolean(false);
-    private final AtomicBoolean consumerEnded = new AtomicBoolean(false);
-
-    public AbstractStreamConsumer(final String id) {
-        uniqueId = id;
-    }
-
-    @Override
-    public final void setReturnBufferQueue(final BufferPool returnQueue) {
-        bufferPool = returnQueue;
-    }
-
-    @Override
-    public final void addFilledBuffer(final ByteBuffer buffer) {
-        if (isConsumerFinished()) {
-            buffer.clear();
-            bufferPool.returnBuffer(buffer, buffer.remaining());
-        } else {
-            filledBuffers.add(buffer);
-        }
-    }
-
-    @Override
-    public final void process() throws IOException {
-        if (isConsumerFinished()) {
-            return;
-        }
-        if (streamEnded.get() && filledBuffers.isEmpty()) {
-            consumerEnded.set(true);
-            onConsumerDone();
-            return;
-        }
-        final ByteBuffer buffer = filledBuffers.poll();
-        if (buffer != null) {
-            final int bytesToProcess = buffer.remaining();
-            try {
-                processBuffer(buffer);
-            } finally {
-                buffer.clear();
-                bufferPool.returnBuffer(buffer, bytesToProcess);
-            }
-        }
-    }
-
-    protected abstract void processBuffer(ByteBuffer buffer) throws IOException;
-
-    @Override
-    public final void signalEndOfStream() {
-        streamEnded.set(true);
-    }
-
-    /**
-     * Convenience method that is called when the consumer is done processing
-     * based on being told the signal is end of stream and has processed all
-     * given buffers.
-     */
-    protected void onConsumerDone() {
-    }
-
-    @Override
-    public final boolean isConsumerFinished() {
-        return consumerEnded.get();
-    }
-
-    @Override
-    public final String getId() {
-        return uniqueId;
-    }
-
-    @Override
-    public final boolean equals(final Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (obj == this) {
-            return true;
-        }
-        if (obj.getClass() != getClass()) {
-            return false;
-        }
-        AbstractStreamConsumer rhs = (AbstractStreamConsumer) obj;
-        return new EqualsBuilder().appendSuper(super.equals(obj)).append(uniqueId, rhs.uniqueId).isEquals();
-    }
-
-    @Override
-    public final int hashCode() {
-        return new HashCodeBuilder(19, 23).append(uniqueId).toHashCode();
-    }
-
-    @Override
-    public final String toString() {
-        return new ToStringBuilder(this, ToStringStyle.NO_FIELD_NAMES_STYLE).append(uniqueId).toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/StreamConsumer.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/StreamConsumer.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/StreamConsumer.java
deleted file mode 100644
index d75b7d7..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/StreamConsumer.java
+++ /dev/null
@@ -1,80 +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.io.nio.consumer;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import org.apache.nifi.io.nio.BufferPool;
-
-/**
- * A StreamConsumer must be thread safe. It may be accessed concurrently by a
- * thread providing data to process and another thread that is processing that
- * data.
- *
- * @author none
- */
-public interface StreamConsumer {
-
-    /**
-     * Will be called once just after construction. It provides the queue to
-     * which processed and emptied and cleared buffers must be returned. For
-     * each time <code>addFilledBuffer</code> is called there should be an
-     * associated add to this given queue. If not, buffers will run out and all
-     * stream processing will halt. READ THIS!!!
-     *
-     * @param returnQueue
-     */
-    void setReturnBufferQueue(BufferPool returnQueue);
-
-    /**
-     * Will be called by the thread that produces byte buffers with available
-     * data to be processed. If the consumer is finished this should simply
-     * return the given buffer to the return buffer queue (after it is cleared)
-     *
-     * @param buffer
-     */
-    void addFilledBuffer(ByteBuffer buffer);
-
-    /**
-     * Will be called by the thread that executes the consumption of data. May
-     * be called many times though once <code>isConsumerFinished</code> returns
-     * true this method will likely do nothing.
-     * @throws java.io.IOException
-     */
-    void process() throws IOException;
-
-    /**
-     * Called once the end of the input stream is detected
-     */
-    void signalEndOfStream();
-
-    /**
-     * If true signals the consumer is done consuming data and will not process
-     * any more buffers.
-     *
-     * @return
-     */
-    boolean isConsumerFinished();
-
-    /**
-     * Uniquely identifies the consumer
-     *
-     * @return
-     */
-    String getId();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/StreamConsumerFactory.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/StreamConsumerFactory.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/StreamConsumerFactory.java
deleted file mode 100644
index df298d5..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/StreamConsumerFactory.java
+++ /dev/null
@@ -1,27 +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.io.nio.consumer;
-
-/**
- *
- * @author none
- */
-public interface StreamConsumerFactory {
-
-    StreamConsumer newInstance(String streamId);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SSLContextFactory.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SSLContextFactory.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SSLContextFactory.java
deleted file mode 100644
index 7ed5ad4..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SSLContextFactory.java
+++ /dev/null
@@ -1,102 +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.io.socket;
-
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.security.KeyManagementException;
-import java.security.KeyStore;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.SecureRandom;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-
-import javax.net.ssl.KeyManager;
-import javax.net.ssl.KeyManagerFactory;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.TrustManager;
-import javax.net.ssl.TrustManagerFactory;
-
-import org.apache.nifi.util.NiFiProperties;
-
-public class SSLContextFactory {
-
-    private final String keystore;
-    private final char[] keystorePass;
-    private final String keystoreType;
-    private final String truststore;
-    private final char[] truststorePass;
-    private final String truststoreType;
-
-    private final KeyManager[] keyManagers;
-    private final TrustManager[] trustManagers;
-
-    public SSLContextFactory(final NiFiProperties properties) throws NoSuchAlgorithmException, CertificateException, FileNotFoundException, IOException, KeyStoreException, UnrecoverableKeyException {
-        keystore = properties.getProperty(NiFiProperties.SECURITY_KEYSTORE);
-        keystorePass = getPass(properties.getProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD));
-        keystoreType = properties.getProperty(NiFiProperties.SECURITY_KEYSTORE_TYPE);
-
-        truststore = properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE);
-        truststorePass = getPass(properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD));
-        truststoreType = properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_TYPE);
-
-        // prepare the keystore
-        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
-        keyStore.load(new FileInputStream(keystore), keystorePass);
-        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
-        keyManagerFactory.init(keyStore, keystorePass);
-
-        // prepare the truststore
-        final KeyStore trustStore = KeyStore.getInstance(truststoreType);
-        trustStore.load(new FileInputStream(truststore), truststorePass);
-        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
-        trustManagerFactory.init(trustStore);
-
-        keyManagers = keyManagerFactory.getKeyManagers();
-        trustManagers = trustManagerFactory.getTrustManagers();
-    }
-
-    private static char[] getPass(final String password) {
-        return password == null ? null : password.toCharArray();
-    }
-
-    /**
-     * Creates a SSLContext instance using the given information.
-     *
-     *
-     * @return a SSLContext instance
-     * @throws java.security.KeyStoreException
-     * @throws java.io.IOException
-     * @throws java.security.NoSuchAlgorithmException
-     * @throws java.security.cert.CertificateException
-     * @throws java.security.UnrecoverableKeyException
-     * @throws java.security.KeyManagementException
-     */
-    public SSLContext createSslContext() throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
-            UnrecoverableKeyException, KeyManagementException {
-
-        // initialize the ssl context
-        final SSLContext sslContext = SSLContext.getInstance("TLS");
-        sslContext.init(keyManagers, trustManagers, new SecureRandom());
-        sslContext.getDefaultSSLParameters().setNeedClientAuth(true);
-
-        return sslContext;
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/ServerSocketConfiguration.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/ServerSocketConfiguration.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/ServerSocketConfiguration.java
deleted file mode 100644
index fc279fb..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/ServerSocketConfiguration.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.io.socket;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.security.KeyManagementException;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-
-import javax.net.ssl.SSLContext;
-
-/**
- * @author unattributed
- */
-public final class ServerSocketConfiguration {
-
-    private boolean needClientAuth;
-    private Integer socketTimeout;
-    private Boolean reuseAddress;
-    private Integer receiveBufferSize;
-    private SSLContextFactory sslContextFactory;
-
-    public ServerSocketConfiguration() {
-    }
-
-    public SSLContext createSSLContext() throws KeyManagementException, NoSuchAlgorithmException, UnrecoverableKeyException, KeyStoreException, CertificateException, FileNotFoundException, IOException {
-        return sslContextFactory == null ? null : sslContextFactory.createSslContext();
-    }
-
-    public void setSSLContextFactory(final SSLContextFactory sslContextFactory) {
-        this.sslContextFactory = sslContextFactory;
-    }
-
-    public Integer getSocketTimeout() {
-        return socketTimeout;
-    }
-
-    public void setSocketTimeout(Integer socketTimeout) {
-        this.socketTimeout = socketTimeout;
-    }
-
-    public boolean getNeedClientAuth() {
-        return needClientAuth;
-    }
-
-    public void setNeedClientAuth(boolean needClientAuth) {
-        this.needClientAuth = needClientAuth;
-    }
-
-    public Boolean getReuseAddress() {
-        return reuseAddress;
-    }
-
-    public void setReuseAddress(Boolean reuseAddress) {
-        this.reuseAddress = reuseAddress;
-    }
-
-    public Integer getReceiveBufferSize() {
-        return receiveBufferSize;
-    }
-
-    public void setReceiveBufferSize(Integer receiveBufferSize) {
-        this.receiveBufferSize = receiveBufferSize;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketConfiguration.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketConfiguration.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketConfiguration.java
deleted file mode 100644
index c24b540..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketConfiguration.java
+++ /dev/null
@@ -1,116 +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.io.socket;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.security.KeyManagementException;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-
-import javax.net.ssl.SSLContext;
-
-/**
- * @author unattributed
- */
-public final class SocketConfiguration {
-
-    private Integer socketTimeout;
-    private Integer receiveBufferSize;
-    private Integer sendBufferSize;
-    private Boolean reuseAddress;
-    private Boolean keepAlive;
-    private Boolean oobInline;
-    private Boolean tcpNoDelay;
-    private Integer trafficClass;
-    private SSLContextFactory sslContextFactory;
-
-    public SSLContext createSSLContext() throws KeyManagementException, NoSuchAlgorithmException, UnrecoverableKeyException, KeyStoreException, CertificateException, FileNotFoundException, IOException {
-        return sslContextFactory == null ? null : sslContextFactory.createSslContext();
-    }
-
-    public void setSSLContextFactory(final SSLContextFactory sslContextFactory) {
-        this.sslContextFactory = sslContextFactory;
-    }
-
-    public Integer getSocketTimeout() {
-        return socketTimeout;
-    }
-
-    public void setSocketTimeout(Integer socketTimeout) {
-        this.socketTimeout = socketTimeout;
-    }
-
-    public Boolean getReuseAddress() {
-        return reuseAddress;
-    }
-
-    public void setReuseAddress(Boolean reuseAddress) {
-        this.reuseAddress = reuseAddress;
-    }
-
-    public Boolean getKeepAlive() {
-        return keepAlive;
-    }
-
-    public void setKeepAlive(Boolean keepAlive) {
-        this.keepAlive = keepAlive;
-    }
-
-    public Boolean getOobInline() {
-        return oobInline;
-    }
-
-    public void setOobInline(Boolean oobInline) {
-        this.oobInline = oobInline;
-    }
-
-    public Integer getReceiveBufferSize() {
-        return receiveBufferSize;
-    }
-
-    public void setReceiveBufferSize(Integer receiveBufferSize) {
-        this.receiveBufferSize = receiveBufferSize;
-    }
-
-    public Integer getSendBufferSize() {
-        return sendBufferSize;
-    }
-
-    public void setSendBufferSize(Integer sendBufferSize) {
-        this.sendBufferSize = sendBufferSize;
-    }
-
-    public Boolean getTcpNoDelay() {
-        return tcpNoDelay;
-    }
-
-    public void setTcpNoDelay(Boolean tcpNoDelay) {
-        this.tcpNoDelay = tcpNoDelay;
-    }
-
-    public Integer getTrafficClass() {
-        return trafficClass;
-    }
-
-    public void setTrafficClass(Integer trafficClass) {
-        this.trafficClass = trafficClass;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketListener.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketListener.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketListener.java
deleted file mode 100644
index e02791a..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketListener.java
+++ /dev/null
@@ -1,211 +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.io.socket;
-
-import java.io.IOException;
-import java.net.ServerSocket;
-import java.net.Socket;
-import java.net.SocketException;
-import java.net.SocketTimeoutException;
-import java.security.KeyManagementException;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.nifi.logging.NiFiLog;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Implements a listener for TCP/IP messages sent over unicast socket.
- *
- * @author unattributed
- */
-public abstract class SocketListener {
-
-    private static final int DEFAULT_SHUTDOWN_LISTENER_SECONDS = 5;
-    private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(SocketListener.class));
-    private volatile ExecutorService executorService;  // volatile to guarantee most current value is visible
-    private volatile ServerSocket serverSocket;        // volatile to guarantee most current value is visible
-    private final int numThreads;
-    private final int port;
-    private final ServerSocketConfiguration configuration;
-    private final AtomicInteger shutdownListenerSeconds = new AtomicInteger(DEFAULT_SHUTDOWN_LISTENER_SECONDS);
-
-    public SocketListener(
-            final int numThreads,
-            final int port,
-            final ServerSocketConfiguration configuration) {
-
-        if (numThreads <= 0) {
-            throw new IllegalArgumentException("Number of threads may not be less than or equal to zero.");
-        } else if (configuration == null) {
-            throw new IllegalArgumentException("Server socket configuration may not be null.");
-        }
-
-        this.numThreads = numThreads;
-        this.port = port;
-        this.configuration = configuration;
-    }
-
-    /**
-     * Implements the action to perform when a new socket request is received.
-     * This class will close the socket.
-     *
-     * @param socket the socket
-     */
-    public abstract void dispatchRequest(final Socket socket);
-
-    public void start() throws IOException {
-
-        if (isRunning()) {
-            return;
-        }
-
-        try {
-            serverSocket = SocketUtils.createServerSocket(port, configuration);
-        } catch (KeyManagementException | UnrecoverableKeyException | NoSuchAlgorithmException | KeyStoreException | CertificateException e) {
-            throw new IOException(e);
-        }
-
-        final ThreadFactory defaultThreadFactory = Executors.defaultThreadFactory();
-        executorService = Executors.newFixedThreadPool(numThreads, new ThreadFactory() {
-            private final AtomicLong threadCounter = new AtomicLong(0L);
-
-            @Override
-            public Thread newThread(final Runnable r) {
-                final Thread newThread = defaultThreadFactory.newThread(r);
-                newThread.setName("Process NCM Request-" + threadCounter.incrementAndGet());
-                return newThread;
-            }
-        });
-
-        final ExecutorService runnableExecServiceRef = executorService;
-        final ServerSocket runnableServerSocketRef = serverSocket;
-
-        final Thread t = new Thread(new Runnable() {
-            @Override
-            public void run() {
-                while (runnableExecServiceRef.isShutdown() == false) {
-                    Socket socket = null;
-                    try {
-                        try {
-                            socket = runnableServerSocketRef.accept();
-                            if (configuration.getSocketTimeout() != null) {
-                                socket.setSoTimeout(configuration.getSocketTimeout());
-                            }
-                        } catch (final SocketTimeoutException ste) {
-                            // nobody connected to us. Go ahead and call closeQuietly just to make sure we don't leave
-                            // any sockets lingering
-                            SocketUtils.closeQuietly(socket);
-                            continue;
-                        } catch (final SocketException se) {
-                            logger.warn("Failed to communicate with " + (socket == null ? "Unknown Host" : socket.getInetAddress().getHostName()) + " due to " + se, se);
-                            SocketUtils.closeQuietly(socket);
-                            continue;
-                        } catch (final Throwable t) {
-                            logger.warn("Socket Listener encountered exception: " + t, t);
-                            SocketUtils.closeQuietly(socket);
-                            continue;
-                        }
-
-                        final Socket finalSocket = socket;
-                        runnableExecServiceRef.execute(new Runnable() {
-                            @Override
-                            public void run() {
-                                try {
-                                    dispatchRequest(finalSocket);
-                                } catch (final Throwable t) {
-                                    logger.warn("Dispatching socket request encountered exception due to: " + t, t);
-                                } finally {
-                                    SocketUtils.closeQuietly(finalSocket);
-                                }
-                            }
-                        });
-                    } catch (final Throwable t) {
-                        logger.error("Socket Listener encountered exception: " + t, t);
-                        SocketUtils.closeQuietly(socket);
-                    }
-                }
-            }
-        });
-        t.setName("Cluster Socket Listener");
-        t.start();
-    }
-
-    public boolean isRunning() {
-        return (executorService != null && executorService.isShutdown() == false);
-    }
-
-    public void stop() throws IOException {
-
-        if (isRunning() == false) {
-            return;
-        }
-
-        // shutdown executor service
-        try {
-            if (getShutdownListenerSeconds() <= 0) {
-                executorService.shutdownNow();
-            } else {
-                executorService.shutdown();
-            }
-            executorService.awaitTermination(getShutdownListenerSeconds(), TimeUnit.SECONDS);
-        } catch (final InterruptedException ex) {
-            Thread.currentThread().interrupt();
-        } finally {
-            if (executorService.isTerminated()) {
-                logger.info("Socket Listener has been terminated successfully.");
-            } else {
-                logger.warn("Socket Listener has not terminated properly.  There exists an uninterruptable thread that will take an indeterminate amount of time to stop.");
-            }
-        }
-
-        // shutdown server socket
-        SocketUtils.closeQuietly(serverSocket);
-
-    }
-
-    public int getShutdownListenerSeconds() {
-        return shutdownListenerSeconds.get();
-    }
-
-    public void setShutdownListenerSeconds(final int shutdownListenerSeconds) {
-        this.shutdownListenerSeconds.set(shutdownListenerSeconds);
-    }
-
-    public ServerSocketConfiguration getConfiguration() {
-        return configuration;
-    }
-
-    public int getPort() {
-        if (isRunning()) {
-            return serverSocket.getLocalPort();
-        } else {
-            return port;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketUtils.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketUtils.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketUtils.java
deleted file mode 100644
index fb6a00c..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketUtils.java
+++ /dev/null
@@ -1,169 +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.io.socket;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.ServerSocket;
-import java.net.Socket;
-import java.security.KeyManagementException;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLServerSocket;
-import javax.net.ssl.SSLSocket;
-
-import org.apache.nifi.logging.NiFiLog;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author unattributed
- */
-public final class SocketUtils {
-
-    private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(SocketUtils.class));
-
-    public static Socket createSocket(final InetSocketAddress address, final SocketConfiguration config) throws IOException {
-        if (address == null) {
-            throw new IllegalArgumentException("Socket address may not be null.");
-        } else if (config == null) {
-            throw new IllegalArgumentException("Configuration may not be null.");
-        }
-
-        final Socket socket;
-
-        final SSLContext sslContext;
-        try {
-            sslContext = config.createSSLContext();
-        } catch (final Exception e) {
-            throw new IOException("Could not create SSLContext", e);
-        }
-
-        if (sslContext == null) {
-            socket = new Socket(address.getHostName(), address.getPort());
-        } else {
-            socket = sslContext.getSocketFactory().createSocket(address.getHostName(), address.getPort());
-        }
-
-        if (config.getSocketTimeout() != null) {
-            socket.setSoTimeout(config.getSocketTimeout());
-        }
-
-        if (config.getReuseAddress() != null) {
-            socket.setReuseAddress(config.getReuseAddress());
-        }
-
-        if (config.getReceiveBufferSize() != null) {
-            socket.setReceiveBufferSize(config.getReceiveBufferSize());
-        }
-
-        if (config.getSendBufferSize() != null) {
-            socket.setSendBufferSize(config.getSendBufferSize());
-        }
-
-        if (config.getTrafficClass() != null) {
-            socket.setTrafficClass(config.getTrafficClass());
-        }
-
-        if (config.getKeepAlive() != null) {
-            socket.setKeepAlive(config.getKeepAlive());
-        }
-
-        if (config.getOobInline() != null) {
-            socket.setOOBInline(config.getOobInline());
-        }
-
-        if (config.getTcpNoDelay() != null) {
-            socket.setTcpNoDelay(config.getTcpNoDelay());
-        }
-
-        return socket;
-    }
-
-    public static ServerSocket createServerSocket(final int port, final ServerSocketConfiguration config) throws IOException, KeyManagementException, UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException, CertificateException {
-        if (config == null) {
-            throw new NullPointerException("Configuration may not be null.");
-        }
-
-        final SSLContext sslContext = config.createSSLContext();
-        final ServerSocket serverSocket;
-        if (sslContext == null) {
-            serverSocket = new ServerSocket(port);
-        } else {
-            serverSocket = sslContext.getServerSocketFactory().createServerSocket(port);
-            ((SSLServerSocket) serverSocket).setNeedClientAuth(config.getNeedClientAuth());
-        }
-
-        if (config.getSocketTimeout() != null) {
-            serverSocket.setSoTimeout(config.getSocketTimeout());
-        }
-
-        if (config.getReuseAddress() != null) {
-            serverSocket.setReuseAddress(config.getReuseAddress());
-        }
-
-        if (config.getReceiveBufferSize() != null) {
-            serverSocket.setReceiveBufferSize(config.getReceiveBufferSize());
-        }
-
-        return serverSocket;
-    }
-
-    public static void closeQuietly(final Socket socket) {
-        if (socket == null) {
-            return;
-        }
-
-        try {
-            try {
-                // can't shudown input/output individually with secure sockets
-                if ((socket instanceof SSLSocket) == false) {
-                    if (socket.isInputShutdown() == false) {
-                        socket.shutdownInput();
-                    }
-                    if (socket.isOutputShutdown() == false) {
-                        socket.shutdownOutput();
-                    }
-                }
-            } finally {
-                if (socket.isClosed() == false) {
-                    socket.close();
-                }
-            }
-        } catch (final Exception ex) {
-            logger.debug("Failed to close socket due to: " + ex, ex);
-        }
-    }
-
-    public static void closeQuietly(final ServerSocket serverSocket) {
-        if (serverSocket == null) {
-            return;
-        }
-
-        try {
-            serverSocket.close();
-        } catch (final Exception ex) {
-            logger.debug("Failed to close server socket due to: " + ex, ex);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/DiscoverableService.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/DiscoverableService.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/DiscoverableService.java
deleted file mode 100644
index 7a62813..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/DiscoverableService.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.io.socket.multicast;
-
-import java.net.InetSocketAddress;
-
-/**
- * A service that may be discovered at runtime. A service is defined as having a
- * unique case-sensitive service name and a socket address where it is
- * available.
- *
- * @author unattributed
- */
-public interface DiscoverableService {
-
-    /**
-     * The service's name. Two services are considered equal if they have the
-     * same case sensitive service name.
-     *
-     * @return the service's name
-     */
-    String getServiceName();
-
-    /**
-     * @return the service's address
-     */
-    InetSocketAddress getServiceAddress();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/DiscoverableServiceImpl.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/DiscoverableServiceImpl.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/DiscoverableServiceImpl.java
deleted file mode 100644
index 5f378b9..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/DiscoverableServiceImpl.java
+++ /dev/null
@@ -1,78 +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.io.socket.multicast;
-
-import java.net.InetSocketAddress;
-import org.apache.commons.lang3.StringUtils;
-
-/**
- * A basic implementation of the DiscoverableService interface. To services are
- * considered equal if they have the same case-sensitive service name.
- *
- * @author unattributed
- */
-public class DiscoverableServiceImpl implements DiscoverableService {
-
-    private final String serviceName;
-
-    private final InetSocketAddress serviceAddress;
-
-    public DiscoverableServiceImpl(final String serviceName, final InetSocketAddress serviceAddress) {
-        if (StringUtils.isBlank(serviceName)) {
-            throw new IllegalArgumentException("Service name may not be null or empty.");
-        } else if (serviceAddress == null) {
-            throw new IllegalArgumentException("Service address may not be null.");
-        }
-        this.serviceName = serviceName;
-        this.serviceAddress = serviceAddress;
-    }
-
-    @Override
-    public InetSocketAddress getServiceAddress() {
-        return serviceAddress;
-    }
-
-    @Override
-    public String getServiceName() {
-        return serviceName;
-    }
-
-    @Override
-    public String toString() {
-        return String.format("[Discoverable Service: %s available at %s:%d]", serviceName, serviceAddress.getHostName(), serviceAddress.getPort());
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (!(obj instanceof DiscoverableService)) {
-            return false;
-        }
-        final DiscoverableService other = (DiscoverableService) obj;
-        return !((this.serviceName == null) ? (other.getServiceName() != null) : !this.serviceName.equals(other.getServiceName()));
-    }
-
-    @Override
-    public int hashCode() {
-        int hash = 5;
-        hash = 53 * hash + (this.serviceName != null ? this.serviceName.hashCode() : 0);
-        return hash;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastConfiguration.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastConfiguration.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastConfiguration.java
deleted file mode 100644
index ea0b72a..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastConfiguration.java
+++ /dev/null
@@ -1,99 +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.io.socket.multicast;
-
-/**
- * @author unattributed
- */
-public final class MulticastConfiguration {
-
-    private MulticastTimeToLive ttl = DEFAULT_MULTICAST_TTL;
-
-    private Integer socketTimeout;
-
-    private Integer receiveBufferSize;
-
-    private Integer sendBufferSize;
-
-    private Boolean reuseAddress;
-
-    private Integer trafficClass;
-
-    private Boolean loopbackMode;
-
-    public static final MulticastTimeToLive DEFAULT_MULTICAST_TTL = MulticastTimeToLive.SAME_SUBNET;
-
-    public MulticastTimeToLive getTtl() {
-        return ttl;
-    }
-
-    public void setTtl(final MulticastTimeToLive ttl) {
-        if (ttl == null) {
-            throw new NullPointerException("Multicast TTL may not be null.");
-        }
-        this.ttl = ttl;
-    }
-
-    public Integer getSocketTimeout() {
-        return socketTimeout;
-    }
-
-    public void setSocketTimeout(Integer socketTimeout) {
-        this.socketTimeout = socketTimeout;
-    }
-
-    public Boolean getReuseAddress() {
-        return reuseAddress;
-    }
-
-    public void setReuseAddress(Boolean reuseAddress) {
-        this.reuseAddress = reuseAddress;
-    }
-
-    public Integer getReceiveBufferSize() {
-        return receiveBufferSize;
-    }
-
-    public void setReceiveBufferSize(Integer receiveBufferSize) {
-        this.receiveBufferSize = receiveBufferSize;
-    }
-
-    public Integer getSendBufferSize() {
-        return sendBufferSize;
-    }
-
-    public void setSendBufferSize(Integer sendBufferSize) {
-        this.sendBufferSize = sendBufferSize;
-    }
-
-    public Integer getTrafficClass() {
-        return trafficClass;
-    }
-
-    public void setTrafficClass(Integer trafficClass) {
-        this.trafficClass = trafficClass;
-    }
-
-    public Boolean getLoopbackMode() {
-        return loopbackMode;
-    }
-
-    public void setLoopbackMode(Boolean loopbackMode) {
-        this.loopbackMode = loopbackMode;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastListener.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastListener.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastListener.java
deleted file mode 100644
index e562c25..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastListener.java
+++ /dev/null
@@ -1,193 +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.io.socket.multicast;
-
-import java.io.IOException;
-import java.net.DatagramPacket;
-import java.net.InetSocketAddress;
-import java.net.MulticastSocket;
-import java.net.SocketException;
-import java.net.SocketTimeoutException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Implements a listener for protocol messages sent over multicast. If a message
- * is of type MulticastProtocolMessage, then the underlying protocol message is
- * passed to the handler. If the receiving handler produces a message response,
- * then the message is wrapped with a MulticastProtocolMessage before being sent
- * to the originator.
- *
- * @author unattributed
- */
-public abstract class MulticastListener {
-
-    // constants
-    private static final int DEFAULT_SHUTDOWN_LISTENER_SECONDS = 5;
-    private static final int DEFAULT_MAX_PACKET_SIZE_BYTES = 512;
-
-    private static final Logger logger = new org.apache.nifi.logging.NiFiLog(LoggerFactory.getLogger(MulticastListener.class));
-
-    // immutable members
-    private final int numThreads;
-    private final InetSocketAddress multicastAddress;
-    private final MulticastConfiguration configuration;
-
-    private volatile ExecutorService executorService;     // volatile to guarantee most current value is visible
-    private volatile MulticastSocket multicastSocket;     // volatile to guarantee most current value is visible
-
-    private int shutdownListenerSeconds = DEFAULT_SHUTDOWN_LISTENER_SECONDS;
-    private int maxPacketSizeBytes = DEFAULT_MAX_PACKET_SIZE_BYTES;
-
-    public MulticastListener(
-            final int numThreads,
-            final InetSocketAddress multicastAddress,
-            final MulticastConfiguration configuration) {
-
-        if (numThreads <= 0) {
-            throw new IllegalArgumentException("Number of threads may not be less than or equal to zero.");
-        } else if (multicastAddress == null) {
-            throw new IllegalArgumentException("Multicast address may not be null.");
-        } else if (multicastAddress.getAddress().isMulticastAddress() == false) {
-            throw new IllegalArgumentException("Multicast group must be a Class D address.");
-        } else if (configuration == null) {
-            throw new IllegalArgumentException("Multicast configuration may not be null.");
-        }
-
-        this.numThreads = numThreads;
-        this.multicastAddress = multicastAddress;
-        this.configuration = configuration;
-    }
-
-    /**
-     * Implements the action to perform when a new datagram is received. This
-     * class must not close the multicast socket.
-     *
-     * @param multicastSocket
-     * @param packet the datagram socket
-     */
-    public abstract void dispatchRequest(final MulticastSocket multicastSocket, final DatagramPacket packet);
-
-    public void start() throws IOException {
-
-        if (isRunning()) {
-            return;
-        }
-
-        multicastSocket = MulticastUtils.createMulticastSocket(multicastAddress.getPort(), configuration);
-        multicastSocket.joinGroup(multicastAddress.getAddress());
-
-        executorService = Executors.newFixedThreadPool(numThreads);
-
-        final ExecutorService runnableExecServiceRef = executorService;
-        final MulticastSocket runnableMulticastSocketRef = multicastSocket;
-
-        new Thread(new Runnable() {
-            @Override
-            public void run() {
-                while (runnableExecServiceRef.isShutdown() == false) {
-                    try {
-                        final byte[] buf = new byte[maxPacketSizeBytes];
-                        final DatagramPacket packet = new DatagramPacket(buf, maxPacketSizeBytes);
-                        runnableMulticastSocketRef.receive(packet);
-                        runnableExecServiceRef.execute(new Runnable() {
-                            @Override
-                            public void run() {
-                                dispatchRequest(multicastSocket, packet);
-                            }
-                        });
-                    } catch (final SocketException | SocketTimeoutException ste) {
-                        /* ignore so that we can accept connections in approximately a non-blocking fashion */
-                    } catch (final Exception e) {
-                        logger.warn("Cluster protocol receiver encountered exception: " + e, e);
-                    }
-                }
-            }
-        }).start();
-    }
-
-    public boolean isRunning() {
-        return (executorService != null && executorService.isShutdown() == false);
-    }
-
-    public void stop() throws IOException {
-
-        if (isRunning() == false) {
-            return;
-        }
-
-        // shutdown executor service
-        try {
-            if (getShutdownListenerSeconds() <= 0) {
-                executorService.shutdownNow();
-            } else {
-                executorService.shutdown();
-            }
-            executorService.awaitTermination(getShutdownListenerSeconds(), TimeUnit.SECONDS);
-        } catch (final InterruptedException ex) {
-            Thread.currentThread().interrupt();
-        } finally {
-            if (executorService.isTerminated()) {
-                logger.info("Multicast Listener has been terminated successfully.");
-            } else {
-                logger.warn("Multicast Listener has not terminated properly.  There exists an uninterruptable thread that will take an indeterminate amount of time to stop.");
-            }
-        }
-
-        // shutdown server socket
-        if (multicastSocket.isClosed() == false) {
-            multicastSocket.leaveGroup(multicastAddress.getAddress());
-            multicastSocket.close();
-        }
-
-    }
-
-    public int getShutdownListenerSeconds() {
-        return shutdownListenerSeconds;
-    }
-
-    public void setShutdownListenerSeconds(final int shutdownListenerSeconds) {
-        this.shutdownListenerSeconds = shutdownListenerSeconds;
-    }
-
-    public int getMaxPacketSizeBytes() {
-        return maxPacketSizeBytes;
-    }
-
-    public void setMaxPacketSizeBytes(int maxPacketSizeBytes) {
-        if (maxPacketSizeBytes <= 0) {
-            throw new IllegalArgumentException("Max packet size must be greater than zero bytes.");
-        }
-        this.maxPacketSizeBytes = maxPacketSizeBytes;
-    }
-
-    public MulticastConfiguration getConfiguration() {
-        return configuration;
-    }
-
-    public InetSocketAddress getMulticastAddress() {
-        return multicastAddress;
-    }
-
-    public int getNumThreads() {
-        return numThreads;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastServiceDiscovery.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastServiceDiscovery.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastServiceDiscovery.java
deleted file mode 100644
index c254c11..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastServiceDiscovery.java
+++ /dev/null
@@ -1,34 +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.io.socket.multicast;
-
-import java.net.InetSocketAddress;
-
-/**
- * Defines the interface for discovering services based on name. Services are
- * expected to be exposed via socket address and port.
- *
- * @author unattributed
- */
-public interface MulticastServiceDiscovery extends ServiceDiscovery {
-
-    /**
-     * @return the multicast address
-     */
-    InetSocketAddress getMulticastAddress();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastServicesBroadcaster.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastServicesBroadcaster.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastServicesBroadcaster.java
deleted file mode 100644
index a3cff9b..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastServicesBroadcaster.java
+++ /dev/null
@@ -1,33 +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.io.socket.multicast;
-
-import java.net.InetSocketAddress;
-
-/**
- * Defines the interface for broadcasting a service via multicast.
- *
- * @author unattributed
- */
-public interface MulticastServicesBroadcaster extends ServicesBroadcaster {
-
-    /**
-     * @return the multicast address
-     */
-    InetSocketAddress getMulticastAddress();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastTimeToLive.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastTimeToLive.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastTimeToLive.java
deleted file mode 100644
index dad1173..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastTimeToLive.java
+++ /dev/null
@@ -1,50 +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.io.socket.multicast;
-
-/**
- * @author unattributed
- */
-public enum MulticastTimeToLive {
-
-    SAME_HOST(0),
-    SAME_SUBNET(1),
-    SAME_SITE(32),
-    SAME_REGION(64),
-    SAME_CONTINENT(128),
-    UNRESTRICTED(255);
-
-    private final int ttl;
-
-    MulticastTimeToLive(final int ttl) {
-        this.ttl = ttl;
-    }
-
-    public int getTtl() {
-        return ttl;
-    }
-
-    public MulticastTimeToLive valueOfByTtl(final int ttl) {
-        for (final MulticastTimeToLive value : values()) {
-            if (value.getTtl() == ttl) {
-                return value;
-            }
-        }
-        return null;
-    }
-
-}


[48/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/flowfile-packager/src/main/java/org/apache/nifi/util/Unpackage.java
----------------------------------------------------------------------
diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/Unpackage.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/Unpackage.java
deleted file mode 100644
index 19f702c..0000000
--- a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/Unpackage.java
+++ /dev/null
@@ -1,119 +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.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.util.Map;
-import java.util.UUID;
-
-public class Unpackage {
-
-    private static void printUsage() {
-        System.out.println("java " + Unpackage.class.getCanonicalName() + " <version> <input file 1> [<input file 2> <input file 3> ... <input file N>]");
-        System.out.println("<version> : The version of the FlowFile Package format. Valid values are 1, 2, 3");
-        System.out.println("<input file X> : The FlowFile package to unpack");
-        System.out.println();
-    }
-
-    public static void main(final String[] args) throws IOException {
-        if (args.length < 2) {
-            printUsage();
-            return;
-        }
-
-        final String version = args[0];
-
-        int inputFileCount = 0;
-        int outputFileCount = 0;
-
-        for (int i = 1; i < args.length; i++) {
-            final String filename = args[i];
-            final File inFile = new File(filename);
-
-            if (inFile.isDirectory()) {
-                System.out.println("WARNING: input file " + inFile + " is a directory; skipping");
-                continue;
-            }
-
-            if (!inFile.exists() || !inFile.canRead()) {
-                System.out.println("ERROR: unable to read file " + inFile);
-                continue;
-            }
-
-            final File outputDir = new File(inFile.getAbsolutePath() + ".unpacked");
-            if (!outputDir.exists() && !outputDir.mkdir()) {
-                System.out.println("ERROR: Unable to create directory " + outputDir);
-                continue;
-            }
-
-            final File tempFile = new File(outputDir, ".temp." + UUID.randomUUID().toString() + ".unpackage");
-            inputFileCount++;
-            try (final FileInputStream fis = new FileInputStream(inFile);
-                    final BufferedInputStream bufferedIn = new BufferedInputStream(fis)) {
-
-                final FlowFileUnpackager unpackager = createUnpackager(version);
-                while (unpackager.hasMoreData()) {
-                    outputFileCount++;
-                    final Map<String, String> attributes;
-
-                    try (final FileOutputStream fos = new FileOutputStream(tempFile);
-                            final BufferedOutputStream bufferedOut = new BufferedOutputStream(fos)) {
-                        attributes = unpackager.unpackageFlowFile(bufferedIn, bufferedOut);
-                    }
-
-                    String outputFilename = attributes.get("filename");
-                    if (outputFilename == null) {
-                        outputFilename = attributes.get("nf.file.name");
-                    }
-
-                    final File outputFile = new File(outputDir, outputFilename);
-                    tempFile.renameTo(outputFile);
-
-                    final File attributeFilename = new File(outputDir, outputFilename + ".attributes");
-                    try (final FileOutputStream fos = new FileOutputStream(attributeFilename);
-                            final BufferedOutputStream bufferedOut = new BufferedOutputStream(fos)) {
-
-                        for (final Map.Entry<String, String> entry : attributes.entrySet()) {
-                            bufferedOut.write((entry.getKey() + "=" + entry.getValue() + "\n").getBytes("UTF-8"));
-                        }
-                    }
-                }
-            }
-        }
-
-        System.out.println("Unpacked " + inputFileCount + " packages into " + outputFileCount + " files");
-    }
-
-    public static FlowFileUnpackager createUnpackager(final String version) {
-        switch (version) {
-            case "1":
-                return new FlowFileUnpackagerV1();
-            case "2":
-                return new FlowFileUnpackagerV2();
-            case "3":
-                return new FlowFileUnpackagerV3();
-            default:
-                System.out.println("ERROR: Invalid version: " + version + "; must be 1, 2, or 3");
-                return null;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/flowfile-packager/src/test/java/org/apache/nifi/util/TestPackageUnpackageV3.java
----------------------------------------------------------------------
diff --git a/commons/flowfile-packager/src/test/java/org/apache/nifi/util/TestPackageUnpackageV3.java b/commons/flowfile-packager/src/test/java/org/apache/nifi/util/TestPackageUnpackageV3.java
deleted file mode 100644
index 24cd374..0000000
--- a/commons/flowfile-packager/src/test/java/org/apache/nifi/util/TestPackageUnpackageV3.java
+++ /dev/null
@@ -1,56 +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 static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.junit.Test;
-
-public class TestPackageUnpackageV3 {
-
-    @Test
-    public void test() throws IOException {
-        final FlowFilePackager packager = new FlowFilePackagerV3();
-        final FlowFileUnpackager unpackager = new FlowFileUnpackagerV3();
-
-        final byte[] data = "Hello, World!".getBytes("UTF-8");
-        final Map<String, String> map = new HashMap<>();
-        map.put("abc", "cba");
-
-        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        final ByteArrayInputStream in = new ByteArrayInputStream(data);
-        packager.packageFlowFile(in, baos, map, data.length);
-
-        final byte[] encoded = baos.toByteArray();
-        final ByteArrayInputStream encodedIn = new ByteArrayInputStream(encoded);
-        final ByteArrayOutputStream decodedOut = new ByteArrayOutputStream();
-        final Map<String, String> unpackagedAttributes = unpackager.unpackageFlowFile(encodedIn, decodedOut);
-        final byte[] decoded = decodedOut.toByteArray();
-
-        assertEquals(map, unpackagedAttributes);
-        assertTrue(Arrays.equals(data, decoded));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/pom.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/pom.xml b/commons/nifi-expression-language/pom.xml
deleted file mode 100644
index 196c048..0000000
--- a/commons/nifi-expression-language/pom.xml
+++ /dev/null
@@ -1,56 +0,0 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <!--
-      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.
-    -->
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-commons-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-
-    <artifactId>nifi-expression-language</artifactId>
-    <version>0.0.1-SNAPSHOT</version>
-	
-    <packaging>jar</packaging>
-    <name>NiFi Expression Language</name>
-	
-    <build>
-        <plugins>
-            <plugin>
-                <groupId>org.antlr</groupId>
-                <artifactId>antlr3-maven-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <goals>
-                            <goal>antlr</goal>
-                        </goals>
-                    </execution>
-                </executions>
-            </plugin>
-        </plugins>
-    </build>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.antlr</groupId>
-            <artifactId>antlr-runtime</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-api</artifactId>
-        </dependency>
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g b/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g
deleted file mode 100644
index 10394b9..0000000
--- a/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g
+++ /dev/null
@@ -1,210 +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.
- */
-lexer grammar AttributeExpressionLexer;
-
-@header {
-	package org.apache.nifi.attribute.expression.language.antlr;
-	import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
-}
-
-@rulecatch {
-  catch(final Exception e) {
-    throw new AttributeExpressionLanguageParsingException(e);
-  }
-}
-
-@members {
-  public void displayRecognitionError(String[] tokenNames, RecognitionException e) {
-    final StringBuilder sb = new StringBuilder();
-    if ( e.token == null ) {
-    	sb.append("Unrecognized token ");
-    } else {
-    	sb.append("Unexpected token '").append(e.token.getText()).append("' ");
-    }
-    sb.append("at line ").append(e.line);
-    if ( e.approximateLineInfo ) {
-    	sb.append(" (approximately)");
-    }
-    sb.append(", column ").append(e.charPositionInLine);
-    sb.append(". Query: ").append(e.input.toString());
-    
-    throw new AttributeExpressionLanguageParsingException(sb.toString());
-  }
-
-  public void recover(RecognitionException e) {
-  	final StringBuilder sb = new StringBuilder();
-    if ( e.token == null ) {
-    	sb.append("Unrecognized token ");
-    } else {
-    	sb.append("Unexpected token '").append(e.token.getText()).append("' ");
-    }
-    sb.append("at line ").append(e.line);
-    if ( e.approximateLineInfo ) {
-    	sb.append(" (approximately)");
-    }
-    sb.append(", column ").append(e.charPositionInLine);
-    sb.append(". Query: ").append(e.input.toString());
-    
-    throw new AttributeExpressionLanguageParsingException(sb.toString());
-  } 
-}
-
-
-// PUNCTUATION & SPECIAL CHARACTERS
-WHITESPACE : (' '|'\t'|'\n'|'\r')+ { $channel = HIDDEN; };
-COMMENT : '#' ( ~('\n') )* '\n' { $channel = HIDDEN; };
-
-DOLLAR : '$';
-LPAREN	: '(';
-RPAREN	: ')';
-LBRACE  : '{';
-RBRACE  : '}';
-COLON	: ':';
-COMMA	: ',';
-DOT		: '.';
-SEMICOLON : ';';
-NUMBER	: ('0'..'9')+;
-
-TRUE	: 'true';
-FALSE	: 'false';
-
-//
-// FUNCTION NAMES
-//
-
-// ATTRIBUTE KEY SELECTION FUNCTIONS
-ANY_ATTRIBUTE : 'anyAttribute';
-ANY_MATCHING_ATTRIBUTE : 'anyMatchingAttribute';
-ALL_ATTRIBUTES : 'allAttributes';
-ALL_MATCHING_ATTRIBUTES : 'allMatchingAttributes';
-ANY_DELINEATED_VALUE : 'anyDelineatedValue';
-ALL_DELINEATED_VALUES : 'allDelineatedValues';
-
-// NO-SUBJECT FUNCTIONS
-NEXT_INT	: 'nextInt';
-IP	: 'ip';
-UUID : 'UUID';
-HOSTNAME : 'hostname';	// requires boolean arg: prefer FQDN
-NOW	: 'now';
-
-
-// 0 arg functions
-TO_UPPER : 'toUpper';
-TO_LOWER : 'toLower';
-TO_STRING : 'toString';
-LENGTH : 'length';
-TRIM	: 'trim';
-IS_NULL	: 'isNull';
-IS_EMPTY : 'isEmpty';
-NOT_NULL : 'notNull';
-TO_NUMBER : 'toNumber';
-URL_ENCODE : 'urlEncode';
-URL_DECODE : 'urlDecode';
-NOT : 'not';
-COUNT : 'count';
-
-// 1 arg functions
-SUBSTRING_AFTER	: 'substringAfter';
-SUBSTRING_BEFORE : 'substringBefore';
-SUBSTRING_AFTER_LAST : 'substringAfterLast';
-SUBSTRING_BEFORE_LAST : 'substringBeforeLast';
-STARTS_WITH : 'startsWith';
-ENDS_WITH : 'endsWith';
-CONTAINS : 'contains';
-PREPEND	: 'prepend';
-APPEND	: 'append';
-INDEX_OF : 'indexOf';
-LAST_INDEX_OF : 'lastIndexOf';
-REPLACE_NULL : 'replaceNull';
-REPLACE_EMPTY : 'replaceEmpty';
-FIND	: 'find';	// regex
-MATCHES : 'matches';	// regex
-EQUALS	: 'equals';
-EQUALS_IGNORE_CASE : 'equalsIgnoreCase';
-GREATER_THAN	: 'gt';
-LESS_THAN		: 'lt';
-GREATER_THAN_OR_EQUAL	: 'ge';
-LESS_THAN_OR_EQUAL		: 'le';
-FORMAT			: 'format'; // takes string date format; uses SimpleDateFormat
-TO_DATE			: 'toDate'; // takes string date format; converts the subject to a Long based on the date format
-MOD : 'mod';
-PLUS : 'plus';
-MINUS : 'minus';
-MULTIPLY : 'multiply';
-DIVIDE : 'divide';
-TO_RADIX : 'toRadix';
-OR : 'or';
-AND : 'and';
-JOIN : 'join';
-
-// 2 arg functions
-SUBSTRING	: 'substring';
-REPLACE	: 'replace';
-REPLACE_ALL : 'replaceAll';
-
-
-// STRINGS
-STRING_LITERAL
-@init{StringBuilder lBuf = new StringBuilder();}
-	:
-		(
-			'"'
-				(
-					escaped=ESC {lBuf.append(getText());} |
-				  	normal = ~( '"' | '\\' | '\n' | '\r' | '\t' ) { lBuf.appendCodePoint(normal);} 
-				)*
-			'"'
-		)
-		{
-			setText(lBuf.toString());
-		}
-		|
-		(
-			'\''
-				(
-					escaped=ESC {lBuf.append(getText());} |
-				  	normal = ~( '\'' | '\\' | '\n' | '\r' | '\t' ) { lBuf.appendCodePoint(normal);} 
-				)*
-			'\''
-		)
-		{
-			setText(lBuf.toString());
-		}
-		;
-
-
-fragment
-ESC
-	:	'\\'
-		(
-				'"'		{ setText("\""); }
-			|	'\''	{ setText("\'"); }
-			|	'r'		{ setText("\r"); }
-			|	'n'		{ setText("\n"); }
-			|	't'		{ setText("\t"); }
-			|	'\\'	{ setText("\\\\"); }
-			|	nextChar = ~('"' | '\'' | 'r' | 'n' | 't' | '\\')		
-				{
-					StringBuilder lBuf = new StringBuilder(); lBuf.append("\\\\").appendCodePoint(nextChar); setText(lBuf.toString());
-				}
-		)
-	;
-
-ATTRIBUTE_NAME : (
-				  ~('$' | '{' | '}' | '(' | ')' | '[' | ']' | ',' | ':' | ';' | '/' | '*' | '\'' | ' ' | '\t' | '\r' | '\n' | '0'..'9')
-				  ~('$' | '{' | '}' | '(' | ')' | '[' | ']' | ',' | ':' | ';' | '/' | '*' | '\'' | ' ' | '\t' | '\r' | '\n')*
-				 );

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g b/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g
deleted file mode 100644
index f6a87dd..0000000
--- a/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g
+++ /dev/null
@@ -1,139 +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.
- */
-parser grammar AttributeExpressionParser;
-
-options {
-	output=AST;
-	tokenVocab=AttributeExpressionLexer;
-}
-
-tokens {
-	QUERY;
-	ATTRIBUTE_REFERENCE;
-	ATTR_NAME;
-	FUNCTION_CALL;
-	EXPRESSION;
-	MULTI_ATTRIBUTE_REFERENCE;
-	QUOTED_ATTR_NAME;
-}
-
-@header {
-	package org.apache.nifi.attribute.expression.language.antlr;
-	import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
-}
-
-@members {
-  public void displayRecognitionError(String[] tokenNames, RecognitionException e) {
-  	final StringBuilder sb = new StringBuilder();
-    if ( e.token == null ) {
-    	sb.append("Unrecognized token ");
-    } else {
-    	sb.append("Unexpected token '").append(e.token.getText()).append("' ");
-    }
-    sb.append("at line ").append(e.line);
-    if ( e.approximateLineInfo ) {
-    	sb.append(" (approximately)");
-    }
-    sb.append(", column ").append(e.charPositionInLine);
-    sb.append(". Query: ").append(e.input.toString());
-    
-    throw new AttributeExpressionLanguageParsingException(sb.toString());
-  }
-
-  public void recover(final RecognitionException e) {
-  	final StringBuilder sb = new StringBuilder();
-    if ( e.token == null ) {
-    	sb.append("Unrecognized token ");
-    } else {
-    	sb.append("Unexpected token '").append(e.token.getText()).append("' ");
-    }
-    sb.append("at line ").append(e.line);
-    if ( e.approximateLineInfo ) {
-    	sb.append(" (approximately)");
-    }
-    sb.append(", column ").append(e.charPositionInLine);
-    sb.append(". Query: ").append(e.input.toString());
-    
-    throw new AttributeExpressionLanguageParsingException(sb.toString());
-  } 
-}
-
-// functions that return Strings
-zeroArgString : (TO_UPPER | TO_LOWER | TRIM | TO_STRING | URL_ENCODE | URL_DECODE) LPAREN! RPAREN!;
-oneArgString : ((SUBSTRING_BEFORE | SUBSTRING_BEFORE_LAST | SUBSTRING_AFTER | SUBSTRING_AFTER_LAST | REPLACE_NULL | REPLACE_EMPTY |
-				PREPEND | APPEND | FORMAT | STARTS_WITH | ENDS_WITH | CONTAINS | JOIN) LPAREN! anyArg RPAREN!) |
-			   (TO_RADIX LPAREN! anyArg (COMMA! anyArg)? RPAREN!);
-twoArgString : ((REPLACE | REPLACE_ALL) LPAREN! anyArg COMMA! anyArg RPAREN!) |
-			   (SUBSTRING LPAREN! anyArg (COMMA! anyArg)? RPAREN!);
-
-
-// functions that return Booleans
-zeroArgBool : (IS_NULL | NOT_NULL | IS_EMPTY | NOT) LPAREN! RPAREN!;
-oneArgBool	: ((FIND | MATCHES | EQUALS_IGNORE_CASE) LPAREN! anyArg RPAREN!) |
-			  (GREATER_THAN | LESS_THAN | GREATER_THAN_OR_EQUAL | LESS_THAN_OR_EQUAL) LPAREN! anyArg RPAREN! |
-			  (EQUALS) LPAREN! anyArg RPAREN! |
-			  (AND | OR) LPAREN! anyArg RPAREN!;
-
-
-// functions that return Numbers
-zeroArgNum	: (LENGTH | TO_NUMBER | COUNT) LPAREN! RPAREN!;
-oneArgNum	: ((INDEX_OF | LAST_INDEX_OF) LPAREN! anyArg RPAREN!) |
-			  (TO_DATE LPAREN! anyArg? RPAREN!) |
-			  ((MOD | PLUS | MINUS | MULTIPLY | DIVIDE) LPAREN! anyArg RPAREN!);
-
-stringFunctionRef : zeroArgString | oneArgString | twoArgString;
-booleanFunctionRef : zeroArgBool | oneArgBool;
-numberFunctionRef : zeroArgNum | oneArgNum;
-
-anyArg : NUMBER | numberFunctionRef | STRING_LITERAL | zeroArgString | oneArgString | twoArgString | booleanLiteral | zeroArgBool | oneArgBool | expression;
-stringArg : STRING_LITERAL | zeroArgString | oneArgString | twoArgString | expression;
-functionRef : stringFunctionRef | booleanFunctionRef | numberFunctionRef;
-
-
-
-// Attribute Reference
-subject : attrName | expression;
-attrName : singleAttrName | multiAttrName;
-
-singleAttrRef : ATTRIBUTE_NAME | STRING_LITERAL;
-singleAttrName : singleAttrRef ->
-	^(ATTR_NAME singleAttrRef);
-	
-
-multiAttrFunction : ANY_ATTRIBUTE | ANY_MATCHING_ATTRIBUTE | ALL_ATTRIBUTES | ALL_MATCHING_ATTRIBUTES | ANY_DELINEATED_VALUE | ALL_DELINEATED_VALUES;
-multiAttrName : multiAttrFunction LPAREN stringArg (COMMA stringArg)* RPAREN ->
-	^(MULTI_ATTRIBUTE_REFERENCE multiAttrFunction stringArg*);
-
-attributeRef : subject ->
-	^(ATTRIBUTE_REFERENCE subject);
-	
-
-functionCall : functionRef ->
-	^(FUNCTION_CALL functionRef);
-
-booleanLiteral : TRUE | FALSE;
-zeroArgStandaloneFunction : (IP | UUID | NOW | NEXT_INT | HOSTNAME) LPAREN! RPAREN!;
-oneArgStandaloneFunction : HOSTNAME^ LPAREN! booleanLiteral RPAREN!;
-standaloneFunction : zeroArgStandaloneFunction | oneArgStandaloneFunction;
-
-attributeRefOrFunctionCall	: (attributeRef | standaloneFunction);
-
-expression : DOLLAR LBRACE attributeRefOrFunctionCall (COLON functionCall)* RBRACE ->
-	^(EXPRESSION attributeRefOrFunctionCall functionCall*);
-
-query : expression EOF ->
-	^(QUERY expression);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/output/AttributeExpressionLexer.tokens
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/output/AttributeExpressionLexer.tokens b/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/output/AttributeExpressionLexer.tokens
deleted file mode 100755
index 0265bfb..0000000
--- a/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/output/AttributeExpressionLexer.tokens
+++ /dev/null
@@ -1,88 +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.
- */
-ALL_ATTRIBUTES=4
-ALL_DELINEATED_VALUES=5
-ALL_MATCHING_ATTRIBUTES=6
-AND=7
-ANY_ATTRIBUTE=8
-ANY_DELINEATED_VALUE=9
-ANY_MATCHING_ATTRIBUTE=10
-APPEND=11
-ATTRIBUTE_NAME=12
-CEIL=13
-COLON=14
-COMMA=15
-CONTAINS=16
-DIVIDE=17
-DOLLAR=18
-DOT=19
-ENDS_WITH=20
-EQUALS=21
-EQUALS_IGNORE_CASE=22
-FALSE=23
-FIND=24
-FLOOR=25
-FORMAT=26
-GREATER_THAN=27
-GREATER_THAN_OR_EQUAL=28
-HOSTNAME=29
-INDEX_OF=30
-IP=31
-IS_NULL=32
-LAST_INDEX_OF=33
-LBRACE=34
-LENGTH=35
-LESS_THAN=36
-LESS_THAN_OR_EQUAL=37
-LPAREN=38
-MATCHES=39
-MINUS=40
-MOD=41
-MULTIPLY=42
-NEXT_INT=43
-NOT=44
-NOT_NULL=45
-NOW=46
-NUMBER=47
-OR=48
-PLUS=49
-PREPEND=50
-RBRACE=51
-REPLACE=52
-REPLACE_ALL=53
-REPLACE_NULL=54
-RPAREN=55
-SEMICOLON=56
-STARTS_WITH=57
-STRING_LITERAL=58
-SUBSTRING=59
-SUBSTRING_AFTER=60
-SUBSTRING_AFTER_LAST=61
-SUBSTRING_BEFORE=62
-SUBSTRING_BEFORE_LAST=63
-TO_DATE=64
-TO_LOWER=65
-TO_NUMBER=66
-TO_RADIX=67
-TO_STRING=68
-TO_UPPER=69
-TRIM=70
-TRUE=71
-URL_DECODE=72
-URL_ENCODE=73
-UUID=74
-WHITESPACE=75

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
deleted file mode 100644
index 81da47e..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
+++ /dev/null
@@ -1,62 +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.attribute.expression.language;
-
-import java.util.Map;
-
-import org.apache.nifi.expression.AttributeValueDecorator;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.exception.ProcessException;
-
-public class EmptyPreparedQuery implements PreparedQuery {
-
-    private final String value;
-
-    EmptyPreparedQuery(final String value) {
-        this.value = value;
-    }
-
-    @Override
-    public String evaluateExpressions(final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException {
-        return value;
-    }
-
-    @Override
-    public String evaluateExpressions() throws ProcessException {
-        return value;
-    }
-
-    @Override
-    public String evaluateExpressions(final AttributeValueDecorator decorator) throws ProcessException {
-        return value;
-    }
-
-    @Override
-    public String evaluateExpressions(final FlowFile flowFile) throws ProcessException {
-        return value;
-    }
-
-    @Override
-    public String evaluateExpressions(Map<String, String> attributes) throws ProcessException {
-        return value;
-    }
-
-    @Override
-    public String evaluateExpressions(Map<String, String> attributes, AttributeValueDecorator decorator) throws ProcessException {
-        return value;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
deleted file mode 100644
index e23bcc0..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
+++ /dev/null
@@ -1,71 +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.attribute.expression.language;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
-import org.apache.nifi.expression.AttributeValueDecorator;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.exception.ProcessException;
-
-
-/**
- * An implementation of PreparedQuery that throws an {@link AttributeExpressionLanguageException} when attempting
- * to evaluate the query. This allows a PreparedQuery to be created, even though it can't
- * be evaluated.
- */
-public class InvalidPreparedQuery implements PreparedQuery {
-    private final String query;
-    private final String explanation;
-    
-    public InvalidPreparedQuery(final String query, final String explanation) {
-        this.query = query;
-        this.explanation = explanation;
-    }
-    
-    @Override
-    public String evaluateExpressions(final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException {
-        throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation);
-    }
-
-    @Override
-    public String evaluateExpressions() throws ProcessException {
-        throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation);
-    }
-
-    @Override
-    public String evaluateExpressions(final AttributeValueDecorator decorator) throws ProcessException {
-        throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation);
-    }
-
-    @Override
-    public String evaluateExpressions(final FlowFile flowFile) throws ProcessException {
-        throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation);
-    }
-
-    @Override
-    public String evaluateExpressions(final Map<String, String> attributes) throws ProcessException {
-        throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation);
-    }
-
-    @Override
-    public String evaluateExpressions(final Map<String, String> attributes, final AttributeValueDecorator decorator) throws ProcessException {
-        throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
deleted file mode 100644
index 0d1b2c7..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
+++ /dev/null
@@ -1,39 +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.attribute.expression.language;
-
-import java.util.Map;
-
-import org.apache.nifi.expression.AttributeValueDecorator;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.exception.ProcessException;
-
-public interface PreparedQuery {
-
-    String evaluateExpressions(FlowFile flowFile, AttributeValueDecorator decorator) throws ProcessException;
-
-    String evaluateExpressions() throws ProcessException;
-
-    String evaluateExpressions(AttributeValueDecorator decorator) throws ProcessException;
-
-    String evaluateExpressions(FlowFile flowFile) throws ProcessException;
-
-    String evaluateExpressions(Map<String, String> attributes) throws ProcessException;
-
-    String evaluateExpressions(Map<String, String> attributes, AttributeValueDecorator decorator) throws ProcessException;
-
-}


[13/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
deleted file mode 100644
index 511bb7d..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
+++ /dev/null
@@ -1,3620 +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.cluster.manager.impl;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.net.URI;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Set;
-import java.util.Timer;
-import java.util.TimerTask;
-import java.util.TreeMap;
-import java.util.UUID;
-import java.util.concurrent.CompletionService;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.regex.Pattern;
-
-import javax.net.ssl.SSLContext;
-import javax.ws.rs.HttpMethod;
-import javax.ws.rs.WebApplicationException;
-import javax.ws.rs.core.StreamingOutput;
-import javax.xml.XMLConstants;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.transform.dom.DOMSource;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import javax.xml.validation.Validator;
-
-import org.apache.nifi.admin.service.AuditService;
-import org.apache.nifi.cluster.BulletinsPayload;
-import org.apache.nifi.cluster.ClusterNodeInformation;
-import org.apache.nifi.cluster.HeartbeatPayload;
-import org.apache.nifi.cluster.NodeInformation;
-import org.apache.nifi.cluster.context.ClusterContext;
-import org.apache.nifi.cluster.context.ClusterContextImpl;
-import org.apache.nifi.cluster.event.Event;
-import org.apache.nifi.cluster.event.EventManager;
-import org.apache.nifi.cluster.firewall.ClusterNodeFirewall;
-import org.apache.nifi.cluster.flow.ClusterDataFlow;
-import org.apache.nifi.cluster.flow.DaoException;
-import org.apache.nifi.cluster.flow.DataFlowManagementService;
-import org.apache.nifi.cluster.flow.PersistedFlowState;
-import org.apache.nifi.cluster.manager.HttpClusterManager;
-import org.apache.nifi.cluster.manager.HttpRequestReplicator;
-import org.apache.nifi.cluster.manager.HttpResponseMapper;
-import org.apache.nifi.cluster.manager.NodeResponse;
-import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException;
-import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException;
-import org.apache.nifi.cluster.manager.exception.IllegalClusterStateException;
-import org.apache.nifi.cluster.manager.exception.IllegalNodeDeletionException;
-import org.apache.nifi.cluster.manager.exception.IllegalNodeDisconnectionException;
-import org.apache.nifi.cluster.manager.exception.IllegalNodeReconnectionException;
-import org.apache.nifi.cluster.manager.exception.IneligiblePrimaryNodeException;
-import org.apache.nifi.cluster.manager.exception.NoConnectedNodesException;
-import org.apache.nifi.cluster.manager.exception.NoResponseFromNodesException;
-import org.apache.nifi.cluster.manager.exception.NodeDisconnectionException;
-import org.apache.nifi.cluster.manager.exception.NodeReconnectionException;
-import org.apache.nifi.cluster.manager.exception.PrimaryRoleAssignmentException;
-import org.apache.nifi.cluster.manager.exception.SafeModeMutableRequestException;
-import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
-import org.apache.nifi.cluster.manager.exception.UriConstructionException;
-import org.apache.nifi.cluster.node.Node;
-import org.apache.nifi.cluster.node.Node.Status;
-import org.apache.nifi.cluster.protocol.ConnectionRequest;
-import org.apache.nifi.cluster.protocol.ConnectionResponse;
-import org.apache.nifi.cluster.protocol.Heartbeat;
-import org.apache.nifi.cluster.protocol.NodeBulletins;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.StandardDataFlow;
-import org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderListener;
-import org.apache.nifi.cluster.protocol.impl.ClusterServicesBroadcaster;
-import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage;
-import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage;
-import org.apache.nifi.cluster.protocol.message.ControllerStartupFailureMessage;
-import org.apache.nifi.cluster.protocol.message.DisconnectMessage;
-import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
-import org.apache.nifi.cluster.protocol.message.NodeBulletinsMessage;
-import org.apache.nifi.cluster.protocol.message.PrimaryRoleAssignmentMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
-import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.controller.ControllerService;
-import org.apache.nifi.controller.Heartbeater;
-import org.apache.nifi.controller.ReportingTaskNode;
-import org.apache.nifi.controller.ValidationContextFactory;
-import org.apache.nifi.controller.reporting.ClusteredReportingTaskNode;
-import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
-import org.apache.nifi.controller.reporting.StandardReportingInitializationContext;
-import org.apache.nifi.controller.scheduling.StandardProcessScheduler;
-import org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent;
-import org.apache.nifi.controller.service.ControllerServiceNode;
-import org.apache.nifi.controller.service.ControllerServiceProvider;
-import org.apache.nifi.controller.service.StandardControllerServiceProvider;
-import org.apache.nifi.controller.status.ProcessGroupStatus;
-import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
-import org.apache.nifi.controller.status.history.ComponentStatusRepository;
-import org.apache.nifi.controller.status.history.MetricDescriptor;
-import org.apache.nifi.controller.status.history.StatusHistory;
-import org.apache.nifi.controller.status.history.StatusHistoryUtil;
-import org.apache.nifi.controller.status.history.StatusSnapshot;
-import org.apache.nifi.diagnostics.GarbageCollection;
-import org.apache.nifi.diagnostics.StorageUsage;
-import org.apache.nifi.diagnostics.SystemDiagnostics;
-import org.apache.nifi.encrypt.StringEncryptor;
-import org.apache.nifi.engine.FlowEngine;
-import org.apache.nifi.events.BulletinFactory;
-import org.apache.nifi.events.VolatileBulletinRepository;
-import org.apache.nifi.framework.security.util.SslContextFactory;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.logging.NiFiLog;
-import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.nar.NarCloseable;
-import org.apache.nifi.nar.NarThreadContextClassLoader;
-import org.apache.nifi.processor.StandardValidationContextFactory;
-import org.apache.nifi.remote.RemoteResourceManager;
-import org.apache.nifi.remote.RemoteSiteListener;
-import org.apache.nifi.remote.SocketRemoteSiteListener;
-import org.apache.nifi.remote.protocol.socket.ClusterManagerServerProtocol;
-import org.apache.nifi.reporting.Bulletin;
-import org.apache.nifi.reporting.BulletinRepository;
-import org.apache.nifi.reporting.InitializationException;
-import org.apache.nifi.reporting.ReportingInitializationContext;
-import org.apache.nifi.reporting.ReportingTask;
-import org.apache.nifi.reporting.Severity;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.util.DomUtils;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.web.Revision;
-import org.apache.nifi.web.api.dto.FlowSnippetDTO;
-import org.apache.nifi.web.api.dto.NodeDTO;
-import org.apache.nifi.web.api.dto.ProcessGroupDTO;
-import org.apache.nifi.web.api.dto.ProcessorDTO;
-import org.apache.nifi.web.api.dto.RemoteProcessGroupContentsDTO;
-import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
-import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
-import org.apache.nifi.web.api.dto.provenance.ProvenanceDTO;
-import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO;
-import org.apache.nifi.web.api.dto.provenance.ProvenanceRequestDTO;
-import org.apache.nifi.web.api.dto.provenance.ProvenanceResultsDTO;
-import org.apache.nifi.web.api.dto.status.ClusterStatusHistoryDTO;
-import org.apache.nifi.web.api.dto.status.NodeStatusHistoryDTO;
-import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
-import org.apache.nifi.web.api.dto.status.StatusSnapshotDTO;
-import org.apache.nifi.web.api.entity.FlowSnippetEntity;
-import org.apache.nifi.web.api.entity.ProcessGroupEntity;
-import org.apache.nifi.web.api.entity.ProcessorEntity;
-import org.apache.nifi.web.api.entity.ProcessorsEntity;
-import org.apache.nifi.web.api.entity.ProvenanceEntity;
-import org.apache.nifi.web.api.entity.ProvenanceEventEntity;
-import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
-import org.apache.nifi.web.api.entity.RemoteProcessGroupsEntity;
-import org.apache.nifi.web.util.WebUtils;
-
-import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.w3c.dom.DOMException;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.NodeList;
-import org.xml.sax.SAXException;
-import org.xml.sax.SAXParseException;
-
-import com.sun.jersey.api.client.ClientResponse;
-
-import org.apache.nifi.cluster.protocol.message.ReconnectionFailureMessage;
-
-/**
- * Provides a cluster manager implementation. The manager federates incoming
- * HTTP client requests to the nodes' external API using the HTTP protocol. The
- * manager also communicates with nodes using the nodes' internal socket
- * protocol.
- *
- * The manager's socket address may broadcasted using multicast if a
- * MulticastServiceBroadcaster instance is set on this instance. The manager
- * instance must be started after setting the broadcaster.
- *
- * The manager may be configured with an EventManager for recording noteworthy
- * lifecycle events (e.g., first heartbeat received, node status change).
- *
- * The start() and stop() methods must be called to initialize and stop the
- * instance.
- *
- * @author unattributed
- */
-public class WebClusterManager implements HttpClusterManager, ProtocolHandler, ControllerServiceProvider {
-
-    public static final String ROOT_GROUP_ID_ALIAS = "root";
-    public static final String BULLETIN_CATEGORY = "Clustering";
-
-    private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(WebClusterManager.class));
-    private static final Logger heartbeatLogger = new NiFiLog(LoggerFactory.getLogger("org.apache.nifi.cluster.heartbeat"));
-
-    /**
-     * The HTTP header to store a cluster context. An example of what may be
-     * stored in the context is a node's auditable actions in response to a
-     * cluster request. The cluster context is serialized using Java's
-     * serialization mechanism and hex encoded.
-     */
-    public static final String CLUSTER_CONTEXT_HTTP_HEADER = "X-ClusterContext";
-
-    /**
-     * HTTP Header that stores a unique ID for each request that is replicated
-     * to the nodes. This is used for logging purposes so that request
-     * information, such as timing, can be correlated between the NCM and the
-     * nodes
-     */
-    public static final String REQUEST_ID_HEADER = "X-RequestID";
-
-    /**
-     * The HTTP header that the NCM specifies to ask a node if they are able to
-     * process a given request. The value is always 150-NodeContinue. The node
-     * will respond with 150 CONTINUE if it is able to process the request, 417
-     * EXPECTATION_FAILED otherwise.
-     */
-    public static final String NCM_EXPECTS_HTTP_HEADER = "X-NcmExpects";
-    public static final int NODE_CONTINUE_STATUS_CODE = 150;
-
-    /**
-     * The HTTP header that the NCM specifies to indicate that a node should
-     * invalidate the specified user group. This is done to ensure that user
-     * cache is not stale when an administrator modifies a group through the UI.
-     */
-    public static final String CLUSTER_INVALIDATE_USER_GROUP_HEADER = "X-ClusterInvalidateUserGroup";
-
-    /**
-     * The HTTP header that the NCM specifies to indicate that a node should
-     * invalidate the specified user. This is done to ensure that user cache is
-     * not stale when an administrator modifies a user through the UI.
-     */
-    public static final String CLUSTER_INVALIDATE_USER_HEADER = "X-ClusterInvalidateUser";
-
-    /**
-     * The default number of seconds to respond to a connecting node if the
-     * manager cannot provide it with a current data flow.
-     */
-    private static final int DEFAULT_CONNECTION_REQUEST_TRY_AGAIN_SECONDS = 5;
-
-    public static final String DEFAULT_COMPONENT_STATUS_REPO_IMPLEMENTATION = "org.apache.nifi.controller.status.history.VolatileComponentStatusRepository";
-
-    public static final Pattern PROCESSORS_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/processors");
-    public static final Pattern PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/processors/[a-f0-9\\-]{36}");
-
-    public static final Pattern REMOTE_PROCESS_GROUPS_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/remote-process-groups");
-    public static final Pattern REMOTE_PROCESS_GROUP_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/remote-process-groups/[a-f0-9\\-]{36}");
-
-    public static final Pattern PROCESS_GROUP_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))");
-    public static final Pattern TEMPLATE_INSTANCE_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/template-instance");
-    public static final Pattern FLOW_SNIPPET_INSTANCE_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/snippet-instance");
-
-    public static final String PROVENANCE_URI = "/nifi-api/controller/provenance";
-    public static final Pattern PROVENANCE_QUERY_URI = Pattern.compile("/nifi-api/controller/provenance/[a-f0-9\\-]{36}");
-    public static final Pattern PROVENANCE_EVENT_URI = Pattern.compile("/nifi-api/controller/provenance/events/[0-9]+");
-
-    private final NiFiProperties properties;
-    private final HttpRequestReplicator httpRequestReplicator;
-    private final HttpResponseMapper httpResponseMapper;
-    private final DataFlowManagementService dataFlowManagementService;
-    private final ClusterManagerProtocolSenderListener senderListener;
-    private final StringEncryptor encryptor;
-    private final Queue<Heartbeat> pendingHeartbeats = new ConcurrentLinkedQueue<>();
-    private final ReentrantReadWriteLock resourceRWLock = new ReentrantReadWriteLock();
-    private final ClusterManagerLock readLock = new ClusterManagerLock(resourceRWLock.readLock(), "Read");
-    private final ClusterManagerLock writeLock = new ClusterManagerLock(resourceRWLock.writeLock(), "Write");
-
-    private final Set<Node> nodes = new HashSet<>();
-    private final Set<ReportingTaskNode> reportingTasks = new HashSet<>();
-
-    // null means the dataflow should be read from disk
-    private StandardDataFlow cachedDataFlow = null;
-    private NodeIdentifier primaryNodeId = null;
-    private Revision revision = new Revision(0L, "");
-    private Timer heartbeatMonitor;
-    private Timer heartbeatProcessor;
-    private volatile ClusterServicesBroadcaster servicesBroadcaster = null;
-    private volatile EventManager eventManager = null;
-    private volatile ClusterNodeFirewall clusterFirewall = null;
-    private volatile AuditService auditService = null;
-    private volatile ControllerServiceProvider controllerServiceProvider = null;
-
-    private final RemoteSiteListener remoteSiteListener;
-    private final Integer remoteInputPort;
-    private final Boolean remoteCommsSecure;
-    private final BulletinRepository bulletinRepository;
-    private final String instanceId;
-    private final FlowEngine reportingTaskEngine;
-    private final Map<NodeIdentifier, ComponentStatusRepository> componentMetricsRepositoryMap = new HashMap<>();
-    private final StandardProcessScheduler processScheduler;
-    private final long componentStatusSnapshotMillis;
-
-    public WebClusterManager(final HttpRequestReplicator httpRequestReplicator, final HttpResponseMapper httpResponseMapper,
-            final DataFlowManagementService dataFlowManagementService, final ClusterManagerProtocolSenderListener senderListener,
-            final NiFiProperties properties, final StringEncryptor encryptor) {
-
-        if (httpRequestReplicator == null) {
-            throw new IllegalArgumentException("HttpRequestReplicator may not be null.");
-        } else if (httpResponseMapper == null) {
-            throw new IllegalArgumentException("HttpResponseMapper may not be null.");
-        } else if (dataFlowManagementService == null) {
-            throw new IllegalArgumentException("DataFlowManagementService may not be null.");
-        } else if (senderListener == null) {
-            throw new IllegalArgumentException("ClusterManagerProtocolSenderListener may not be null.");
-        } else if (properties == null) {
-            throw new IllegalArgumentException("NiFiProperties may not be null.");
-        }
-
-        // Ensure that our encryptor/decryptor is properly initialized
-        this.httpRequestReplicator = httpRequestReplicator;
-        this.httpResponseMapper = httpResponseMapper;
-        this.dataFlowManagementService = dataFlowManagementService;
-        this.properties = properties;
-        this.controllerServiceProvider = new StandardControllerServiceProvider();
-        this.bulletinRepository = new VolatileBulletinRepository();
-        this.instanceId = UUID.randomUUID().toString();
-        this.senderListener = senderListener;
-        this.encryptor = encryptor;
-        senderListener.addHandler(this);
-        senderListener.setBulletinRepository(bulletinRepository);
-
-        final String snapshotFrequency = properties.getProperty(NiFiProperties.COMPONENT_STATUS_SNAPSHOT_FREQUENCY, NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY);
-        long snapshotMillis;
-        try {
-            snapshotMillis = FormatUtils.getTimeDuration(snapshotFrequency, TimeUnit.MILLISECONDS);
-        } catch (final Exception e) {
-            snapshotMillis = FormatUtils.getTimeDuration(NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY, TimeUnit.MILLISECONDS);
-        }
-        componentStatusSnapshotMillis = snapshotMillis;
-
-        remoteInputPort = properties.getRemoteInputPort();
-        if (remoteInputPort == null) {
-            remoteSiteListener = null;
-            remoteCommsSecure = null;
-        } else {
-            // Register the ClusterManagerServerProtocol as the appropriate resource for site-to-site Server Protocol
-            RemoteResourceManager.setServerProtocolImplementation(ClusterManagerServerProtocol.RESOURCE_NAME, ClusterManagerServerProtocol.class);
-            remoteCommsSecure = properties.isSiteToSiteSecure();
-            if (remoteCommsSecure) {
-                final SSLContext sslContext = SslContextFactory.createSslContext(properties, false);
-
-                if (sslContext == null) {
-                    throw new IllegalStateException("NiFi Configured to allow Secure Site-to-Site communications but the Keystore/Truststore properties are not configured");
-                }
-
-                remoteSiteListener = new SocketRemoteSiteListener(remoteInputPort.intValue(), sslContext, this);
-            } else {
-                remoteSiteListener = new SocketRemoteSiteListener(remoteInputPort.intValue(), null, this);
-            }
-        }
-
-        reportingTaskEngine = new FlowEngine(8, "Reporting Task Thread");
-
-        processScheduler = new StandardProcessScheduler(new Heartbeater() {
-            @Override
-            public void heartbeat() {
-            }
-        }, this, encryptor);
-        processScheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, new TimerDrivenSchedulingAgent(null, reportingTaskEngine, null, encryptor));
-        processScheduler.setMaxThreadCount(SchedulingStrategy.TIMER_DRIVEN, 10);
-        processScheduler.setMaxThreadCount(SchedulingStrategy.CRON_DRIVEN, 10);
-    }
-
-    public void start() throws IOException {
-        writeLock.lock();
-        try {
-
-            if (isRunning()) {
-                throw new IllegalStateException("Instance is already started.");
-            }
-
-            try {
-                // setup heartbeat monitoring
-                heartbeatMonitor = new Timer("Heartbeat Monitor", /* is daemon */ true);
-                heartbeatMonitor.scheduleAtFixedRate(new HeartbeatMonitoringTimerTask(), 0, getHeartbeatMonitoringIntervalSeconds() * 1000);
-
-                heartbeatProcessor = new Timer("Process Pending Heartbeats", true);
-                final int processPendingHeartbeatDelay = 1000 * Math.max(1, getClusterProtocolHeartbeatSeconds() / 2);
-                heartbeatProcessor.schedule(new ProcessPendingHeartbeatsTask(), processPendingHeartbeatDelay, processPendingHeartbeatDelay);
-
-                // start request replication service
-                httpRequestReplicator.start();
-
-                // start protocol service
-                senderListener.start();
-
-                // start flow management service
-                dataFlowManagementService.start();
-
-                if (remoteSiteListener != null) {
-                    remoteSiteListener.start();
-                }
-
-                // load flow
-                if (dataFlowManagementService.isFlowCurrent()) {
-                    final ClusterDataFlow clusterDataFlow = dataFlowManagementService.loadDataFlow();
-                    cachedDataFlow = clusterDataFlow.getDataFlow();
-                    primaryNodeId = clusterDataFlow.getPrimaryNodeId();
-                } else {
-                    throw new IOException("Flow is not current.");
-                }
-
-                // start multicast broadcasting service, if configured
-                if (servicesBroadcaster != null) {
-                    servicesBroadcaster.start();
-                }
-
-                // start in safe mode
-                executeSafeModeTask();
-
-                // Load and start running Reporting Tasks
-                final File taskFile = new File(properties.getProperty(NiFiProperties.TASK_CONFIGURATION_FILE));
-                reportingTasks.addAll(loadReportingTasks(taskFile));
-            } catch (final IOException ioe) {
-                logger.warn("Failed to initialize cluster services due to: " + ioe, ioe);
-                stop();
-                throw ioe;
-            }
-
-        } finally {
-            writeLock.unlock("START");
-        }
-    }
-
-    public void stop() throws IOException {
-        writeLock.lock();
-        try {
-
-            // returns true if any service is running
-            if (isRunning() == false) {
-                throw new IllegalArgumentException("Instance is already stopped.");
-            }
-
-            boolean encounteredException = false;
-
-            // stop the heartbeat monitoring
-            if (isHeartbeatMonitorRunning()) {
-                heartbeatMonitor.cancel();
-                heartbeatMonitor = null;
-            }
-
-            if (heartbeatProcessor != null) {
-                heartbeatProcessor.cancel();
-                heartbeatProcessor = null;
-            }
-
-            // stop the HTTP request replicator service
-            if (httpRequestReplicator.isRunning()) {
-                httpRequestReplicator.stop();
-            }
-
-            // stop the flow management service
-            if (dataFlowManagementService.isRunning()) {
-                dataFlowManagementService.stop();
-            }
-
-            if (remoteSiteListener != null) {
-                remoteSiteListener.stop();
-            }
-
-            // stop the protocol listener service
-            if (senderListener.isRunning()) {
-                try {
-                    senderListener.stop();
-                } catch (final IOException ioe) {
-                    encounteredException = true;
-                    logger.warn("Failed to shutdown protocol service due to: " + ioe, ioe);
-                }
-            }
-
-            // stop the service broadcaster
-            if (isBroadcasting()) {
-                servicesBroadcaster.stop();
-            }
-
-            if ( processScheduler != null ) {
-                processScheduler.shutdown();
-            }
-            
-            if (encounteredException) {
-                throw new IOException("Failed to shutdown Cluster Manager because one or more cluster services failed to shutdown.  Check the logs for details.");
-            }
-
-        } finally {
-            writeLock.unlock("STOP");
-        }
-    }
-
-    public boolean isRunning() {
-        readLock.lock();
-        try {
-            return isHeartbeatMonitorRunning()
-                    || httpRequestReplicator.isRunning()
-                    || senderListener.isRunning()
-                    || dataFlowManagementService.isRunning()
-                    || isBroadcasting();
-        } finally {
-            readLock.unlock("isRunning");
-        }
-    }
-
-    @Override
-    public boolean canHandle(ProtocolMessage msg) {
-        return MessageType.CONNECTION_REQUEST == msg.getType()
-                || MessageType.HEARTBEAT == msg.getType()
-                || MessageType.CONTROLLER_STARTUP_FAILURE == msg.getType()
-                || MessageType.BULLETINS == msg.getType()
-                || MessageType.RECONNECTION_FAILURE == msg.getType();
-    }
-
-    @Override
-    public ProtocolMessage handle(final ProtocolMessage protocolMessage) throws ProtocolException {
-        switch (protocolMessage.getType()) {
-            case CONNECTION_REQUEST:
-                return handleConnectionRequest((ConnectionRequestMessage) protocolMessage);
-            case HEARTBEAT:
-                final HeartbeatMessage heartbeatMessage = (HeartbeatMessage) protocolMessage;
-
-                final Heartbeat original = heartbeatMessage.getHeartbeat();
-                final NodeIdentifier originalNodeId = original.getNodeIdentifier();
-                final Heartbeat heartbeatWithDn = new Heartbeat(addRequestorDn(originalNodeId, heartbeatMessage.getRequestorDN()), original.isPrimary(), original.isConnected(), original.getPayload());
-
-                handleHeartbeat(heartbeatWithDn);
-                return null;
-            case CONTROLLER_STARTUP_FAILURE:
-                new Thread(new Runnable() {
-                    @Override
-                    public void run() {
-                        handleControllerStartupFailure((ControllerStartupFailureMessage) protocolMessage);
-                    }
-                }, "Handle Controller Startup Failure Message from " + ((ControllerStartupFailureMessage) protocolMessage).getNodeId()).start();
-                return null;
-            case RECONNECTION_FAILURE:
-                new Thread(new Runnable() {
-                    @Override
-                    public void run() {
-                        handleReconnectionFailure((ReconnectionFailureMessage) protocolMessage);
-                    }
-                }, "Handle Reconnection Failure Message from " + ((ReconnectionFailureMessage) protocolMessage).getNodeId()).start();
-                return null;
-            case BULLETINS:
-                final NodeBulletinsMessage bulletinsMessage = (NodeBulletinsMessage) protocolMessage;
-                handleBulletins(bulletinsMessage.getBulletins());
-                return null;
-            default:
-                throw new ProtocolException("No handler defined for message type: " + protocolMessage.getType());
-        }
-    }
-
-    /**
-     * Services connection requests. If the data flow management service is
-     * unable to provide a current copy of the data flow, then the returned
-     * connection response will indicate the node should try later. Otherwise,
-     * the connection response will contain the the flow and the node
-     * identifier.
-     *
-     * If this instance is configured with a firewall and the request is
-     * blocked, then the response will not contain a node identifier.
-     *
-     * @param request a connection request
-     *
-     * @return a connection response
-     */
-    @Override
-    public ConnectionResponse requestConnection(final ConnectionRequest request) {
-        final boolean lockObtained = writeLock.tryLock(3, TimeUnit.SECONDS);
-        if (!lockObtained) {
-            // Create try-later response because we are too busy to service the request right now. We do not want
-            // to wait long because we want Node/NCM comms to be very responsive
-            final int tryAgainSeconds;
-            if (dataFlowManagementService.getRetrievalDelaySeconds() <= 0) {
-                tryAgainSeconds = DEFAULT_CONNECTION_REQUEST_TRY_AGAIN_SECONDS;
-            } else {
-                tryAgainSeconds = dataFlowManagementService.getRetrievalDelaySeconds();
-            }
-
-            // record event
-            final String msg = "Connection requested from node, but manager was too busy to service request.  Instructing node to try again in " + tryAgainSeconds + " seconds.";
-            addEvent(request.getProposedNodeIdentifier(), msg);
-            addBulletin(request.getProposedNodeIdentifier(), Severity.INFO, msg);
-
-            // return try later response
-            return new ConnectionResponse(tryAgainSeconds);
-        }
-
-        try {
-            // resolve the proposed node identifier to a valid node identifier
-            final NodeIdentifier resolvedNodeIdentifier = resolveProposedNodeIdentifier(request.getProposedNodeIdentifier());
-
-            if (isBlockedByFirewall(resolvedNodeIdentifier.getSocketAddress())) {
-                // if the socket address is not listed in the firewall, then return a null response
-                logger.info("Firewall blocked connection request from node " + resolvedNodeIdentifier);
-                return ConnectionResponse.createBlockedByFirewallResponse();
-            }
-
-            // get a raw reference to the node (if it doesn't exist, node will be null)
-            Node node = getRawNode(resolvedNodeIdentifier.getId());
-
-            // create a new node if necessary and set status to connecting
-            if (node == null) {
-                node = new Node(resolvedNodeIdentifier, Status.CONNECTING);
-                addEvent(node.getNodeId(), "Connection requested from new node.  Setting status to connecting.");
-                nodes.add(node);
-            } else {
-                node.setStatus(Status.CONNECTING);
-                addEvent(resolvedNodeIdentifier, "Connection requested from existing node.  Setting status to connecting");
-            }
-
-            // record the time of the connection request
-            node.setConnectionRequestedTimestamp(new Date().getTime());
-
-            // clear out old heartbeat info
-            node.setHeartbeat(null);
-
-            // try to obtain a current flow
-            if (dataFlowManagementService.isFlowCurrent()) {
-                // if a cached copy does not exist, load it from disk
-                if (cachedDataFlow == null) {
-                    final ClusterDataFlow clusterDataFlow = dataFlowManagementService.loadDataFlow();
-                    cachedDataFlow = clusterDataFlow.getDataFlow();
-                    primaryNodeId = clusterDataFlow.getPrimaryNodeId();
-                }
-
-                // determine if this node should be assigned the primary role
-                final boolean primaryRole;
-                if (primaryNodeId == null || primaryNodeId.logicallyEquals(node.getNodeId())) {
-                    setPrimaryNodeId(node.getNodeId());
-                    addEvent(node.getNodeId(), "Setting primary role in connection response.");
-                    primaryRole = true;
-                } else {
-                    primaryRole = false;
-                }
-
-                return new ConnectionResponse(node.getNodeId(), cachedDataFlow, primaryRole, remoteInputPort, remoteCommsSecure, instanceId);
-            }
-
-            /*
-             * The manager does not have a current copy of the data flow, 
-             * so it will instruct the node to try connecting at a later 
-             * time.  Meanwhile, the flow will be locked down from user 
-             * changes because the node is marked as connecting.
-             */
-
-            /*
-             * Create try-later response based on flow retrieval delay to give 
-             * the flow management service a chance to retrieve a curren flow
-             */
-            final int tryAgainSeconds;
-            if (dataFlowManagementService.getRetrievalDelaySeconds() <= 0) {
-                tryAgainSeconds = DEFAULT_CONNECTION_REQUEST_TRY_AGAIN_SECONDS;
-            } else {
-                tryAgainSeconds = dataFlowManagementService.getRetrievalDelaySeconds();
-            }
-
-            // record event
-            addEvent(node.getNodeId(), "Connection requested from node, but manager was unable to obtain current flow.  Instructing node to try again in " + tryAgainSeconds + " seconds.");
-
-            // return try later response
-            return new ConnectionResponse(tryAgainSeconds);
-
-        } finally {
-            writeLock.unlock("requestConnection");
-        }
-    }
-
-    /**
-     * Services reconnection requests for a given node. If the node indicates
-     * reconnection failure, then the node will be set to disconnected and if
-     * the node has primary role, then the role will be revoked. Otherwise, a
-     * reconnection request will be sent to the node, initiating the connection
-     * handshake.
-     *
-     * @param nodeId a node identifier
-     *
-     * @throws UnknownNodeException if the node does not exist
-     * @throws IllegalNodeReconnectionException if the node cannot be
-     * reconnected because the node is not disconnected
-     * @throws NodeReconnectionException if the reconnection message failed to
-     * be sent or the cluster could not provide a current data flow for the
-     * reconnection request
-     */
-    @Override
-    public void requestReconnection(final String nodeId, final String userDn) throws UnknownNodeException, IllegalNodeReconnectionException {
-        Node node = null;
-
-        final boolean primaryRole;
-        final int tryAgainSeconds;
-
-        writeLock.lock();
-        try {
-            // check if we know about this node and that it is disconnected
-            node = getRawNode(nodeId);
-            logger.info("Request was made by {} to reconnect node {} to cluster", userDn, node == null ? nodeId : node);
-
-            if (node == null) {
-                throw new UnknownNodeException("Node does not exist.");
-            } else if (Status.DISCONNECTED != node.getStatus()) {
-                throw new IllegalNodeReconnectionException("Node must be disconnected before it can reconnect.");
-            }
-
-            // clear out old heartbeat info
-            node.setHeartbeat(null);
-
-            // get the dataflow to send with the reconnection request
-            if (!dataFlowManagementService.isFlowCurrent()) {
-                /* node remains disconnected */
-                final String msg = "Reconnection requested for node, but manager was unable to obtain current flow.  Setting node to disconnected.";
-                addEvent(node.getNodeId(), msg);
-                addBulletin(node, Severity.WARNING, msg);
-                throw new NodeReconnectionException("Manager was unable to obtain current flow to provide in reconnection request to node.  Try again in a few seconds.");
-            }
-
-            // if a cached copy does not exist, load it from disk
-            if (cachedDataFlow == null) {
-                final ClusterDataFlow clusterDataFlow = dataFlowManagementService.loadDataFlow();
-                cachedDataFlow = clusterDataFlow.getDataFlow();
-                primaryNodeId = clusterDataFlow.getPrimaryNodeId();
-            }
-
-            node.setStatus(Status.CONNECTING);
-            addEvent(node.getNodeId(), "Reconnection requested for node.  Setting status to connecting.");
-
-            // determine if this node should be assigned the primary role
-            if (primaryNodeId == null || primaryNodeId.logicallyEquals(node.getNodeId())) {
-                setPrimaryNodeId(node.getNodeId());
-                addEvent(node.getNodeId(), "Setting primary role in reconnection request.");
-                primaryRole = true;
-            } else {
-                primaryRole = false;
-            }
-
-            if (dataFlowManagementService.getRetrievalDelaySeconds() <= 0) {
-                tryAgainSeconds = DEFAULT_CONNECTION_REQUEST_TRY_AGAIN_SECONDS;
-            } else {
-                tryAgainSeconds = dataFlowManagementService.getRetrievalDelaySeconds();
-            }
-        } catch (final UnknownNodeException | IllegalNodeReconnectionException | NodeReconnectionException une) {
-            throw une;
-        } catch (final Exception ex) {
-            logger.warn("Problem encountered issuing reconnection request to node " + node.getNodeId() + " due to: " + ex, ex);
-
-            node.setStatus(Status.DISCONNECTED);
-            final String eventMsg = "Problem encountered issuing reconnection request. Node will remain disconnected: " + ex;
-            addEvent(node.getNodeId(), eventMsg);
-            addBulletin(node, Severity.WARNING, eventMsg);
-
-            // Exception thrown will include node ID but event/bulletin do not because the node/id is passed along with the message
-            throw new NodeReconnectionException("Problem encountered issuing reconnection request to " + node.getNodeId() + ". Node will remain disconnected: " + ex, ex);
-        } finally {
-            writeLock.unlock("requestReconnection");
-        }
-
-        // Asynchronously start attempting reconnection. This is not completely thread-safe, as
-        // we do this by releasing the write lock and then obtaining a read lock for each attempt,
-        // so we suffer from the ABA problem. However, we are willing to accept the consequences of
-        // this situation in order to avoid holding a lock for the entire duration. "The consequences"
-        // are that a second thread could potentially be doing the same thing, issuing a reconnection request.
-        // However, this is very unlikely to happen, based on the conditions under which we issue a reconnection
-        // request. And if we do, the node will simply reconnect multiple times, which is not a big deal.
-        requestReconnectionAsynchronously(node, primaryRole, 10, tryAgainSeconds);
-    }
-
-    private void requestReconnectionAsynchronously(final Node node, final boolean primaryRole, final int reconnectionAttempts, final int retrySeconds) {
-        final Thread reconnectionThread = new Thread(new Runnable() {
-            @Override
-            public void run() {
-                for (int i = 0; i < reconnectionAttempts; i++) {
-                    final ReconnectionRequestMessage request = new ReconnectionRequestMessage();
-
-                    try {
-                        readLock.lock();
-                        try {
-                            if (Status.CONNECTING != node.getStatus()) {
-                                // the node status has changed. It's no longer appropriate to attempt reconnection.
-                                return;
-                            }
-
-                            // create the request
-                            request.setNodeId(node.getNodeId());
-                            request.setDataFlow(cachedDataFlow);
-                            request.setPrimary(primaryRole);
-                            request.setManagerRemoteSiteCommsSecure(remoteCommsSecure);
-                            request.setManagerRemoteSiteListeningPort(remoteInputPort);
-                            request.setInstanceId(instanceId);
-                        } finally {
-                            readLock.unlock("Reconnect " + node.getNodeId());
-                        }
-
-                        // Issue a reconnection request to the node.
-                        senderListener.requestReconnection(request);
-
-                        node.setConnectionRequestedTimestamp(System.currentTimeMillis());
-
-                        // successfully told node to reconnect -- we're done!
-                        return;
-                    } catch (final Exception e) {
-                        logger.warn("Problem encountered issuing reconnection request to node " + node.getNodeId() + " due to: " + e);
-                        if (logger.isDebugEnabled()) {
-                            logger.warn("", e);
-                        }
-
-                        addBulletin(node, Severity.WARNING, "Problem encountered issuing reconnection request to node " + node.getNodeId() + " due to: " + e);
-                    }
-
-                    try {
-                        Thread.sleep(1000L * retrySeconds);
-                    } catch (final InterruptedException ie) {
-                        break;
-                    }
-                }
-
-                // We failed to reconnect 10 times. We must now mark node as disconnected.
-                writeLock.lock();
-                try {
-                    if (Status.CONNECTING == node.getStatus()) {
-                        requestDisconnectionQuietly(node.getNodeId(), "Failed to issue Reconnection Request " + reconnectionAttempts + " times");
-                    }
-                } finally {
-                    writeLock.unlock("Mark node as Disconnected as a result of reconnection failure");
-                }
-            }
-        }, "Reconnect " + node.getNodeId());
-
-        reconnectionThread.start();
-    }
-
-    private List<ReportingTaskNode> loadReportingTasks(final File taskConfigXml) {
-        final List<ReportingTaskNode> tasks = new ArrayList<>();
-        if (taskConfigXml == null) {
-            logger.info("No controller tasks to start");
-            return tasks;
-        }
-
-        try {
-            final URL schemaUrl = getClass().getResource("/ReportingTaskConfiguration.xsd");
-            final Document document = parse(taskConfigXml, schemaUrl);
-
-            final NodeList tasksNodes = document.getElementsByTagName("tasks");
-            final Element tasksElement = (Element) tasksNodes.item(0);
-
-            //optional properties for all ReportingTasks
-            for (final Element taskElement : DomUtils.getChildElementsByTagName(tasksElement, "task")) {
-                //add global properties common to all tasks
-                Map<String, String> properties = new HashMap<>();
-
-                //get properties for the specific reporting task - id, name, class,
-                //and schedulingPeriod must be set
-                final String taskId = DomUtils.getChild(taskElement, "id").getTextContent().trim();
-                final String taskName = DomUtils.getChild(taskElement, "name").getTextContent().trim();
-
-                final List<Element> schedulingStrategyNodeList = DomUtils.getChildElementsByTagName(taskElement, "schedulingStrategy");
-                String schedulingStrategyValue = SchedulingStrategy.TIMER_DRIVEN.name();
-                if (schedulingStrategyNodeList.size() == 1) {
-                    final String specifiedValue = schedulingStrategyNodeList.get(0).getTextContent();
-
-                    try {
-                        schedulingStrategyValue = SchedulingStrategy.valueOf(specifiedValue).name();
-                    } catch (final Exception e) {
-                        throw new RuntimeException("Cannot start Reporting Task with id " + taskId + " because its Scheduling Strategy does not have a valid value", e);
-                    }
-                }
-
-                final SchedulingStrategy schedulingStrategy = SchedulingStrategy.valueOf(schedulingStrategyValue);
-                final String taskSchedulingPeriod = DomUtils.getChild(taskElement, "schedulingPeriod").getTextContent().trim();
-                final String taskClass = DomUtils.getChild(taskElement, "class").getTextContent().trim();
-
-                //optional task-specific properties
-                for (final Element optionalProperty : DomUtils.getChildElementsByTagName(taskElement, "property")) {
-                    final String name = optionalProperty.getAttribute("name");
-                    final String value = optionalProperty.getTextContent().trim();
-                    properties.put(name, value);
-                }
-
-                //set the class to be used for the configured reporting task
-                final ReportingTaskNode reportingTaskNode;
-                try {
-                    reportingTaskNode = createReportingTask(taskClass, taskId);
-                } catch (final ReportingTaskInstantiationException e) {
-                    logger.error("Unable to load reporting task {} due to {}", new Object[]{taskId, e});
-                    if (logger.isDebugEnabled()) {
-                        logger.error("", e);
-                    }
-                    continue;
-                }
-
-                final ReportingTask reportingTask = reportingTaskNode.getReportingTask();
-
-                final ReportingInitializationContext config = new StandardReportingInitializationContext(taskId, taskName, schedulingStrategy, taskSchedulingPeriod, this);
-                reportingTask.initialize(config);
-
-                final Map<PropertyDescriptor, String> resolvedProps;
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    resolvedProps = new HashMap<>();
-                    for (final Map.Entry<String, String> entry : properties.entrySet()) {
-                        final PropertyDescriptor descriptor = reportingTask.getPropertyDescriptor(entry.getKey());
-                        resolvedProps.put(descriptor, entry.getValue());
-                    }
-                }
-
-                for (final Map.Entry<PropertyDescriptor, String> entry : resolvedProps.entrySet()) {
-                    reportingTaskNode.setProperty(entry.getKey().getName(), entry.getValue());
-                }
-
-                processScheduler.schedule(reportingTaskNode);
-                tasks.add(reportingTaskNode);
-            }
-        } catch (final SAXException | ParserConfigurationException | IOException | DOMException | NumberFormatException | InitializationException t) {
-            logger.error("Unable to load reporting tasks from {} due to {}", new Object[]{taskConfigXml, t});
-            if (logger.isDebugEnabled()) {
-                logger.error("", t);
-            }
-        }
-
-        return tasks;
-    }
-
-    private ReportingTaskNode createReportingTask(final String type, final String id) throws ReportingTaskInstantiationException {
-        if (type == null) {
-            throw new NullPointerException();
-        }
-        ReportingTask task = null;
-        final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader();
-        try {
-            final ClassLoader detectedClassLoader = ExtensionManager.getClassLoader(type);
-            final Class<?> rawClass;
-            if (detectedClassLoader == null) {
-                rawClass = Class.forName(type);
-            } else {
-                rawClass = Class.forName(type, false, detectedClassLoader);
-            }
-
-            Thread.currentThread().setContextClassLoader(detectedClassLoader);
-            final Class<? extends ReportingTask> reportingTaskClass = rawClass.asSubclass(ReportingTask.class);
-            final Object reportingTaskObj = reportingTaskClass.newInstance();
-            task = reportingTaskClass.cast(reportingTaskObj);
-        } catch (final ClassNotFoundException | SecurityException | InstantiationException | IllegalAccessException | IllegalArgumentException t) {
-            throw new ReportingTaskInstantiationException(type, t);
-        } finally {
-            if (ctxClassLoader != null) {
-                Thread.currentThread().setContextClassLoader(ctxClassLoader);
-            }
-        }
-
-        final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(this);
-        final ReportingTaskNode taskNode = new ClusteredReportingTaskNode(task, id, processScheduler,
-                new ClusteredEventAccess(this), bulletinRepository, controllerServiceProvider, validationContextFactory);
-        return taskNode;
-    }
-
-    private Document parse(final File xmlFile, final URL schemaUrl) throws SAXException, ParserConfigurationException, IOException {
-        final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
-        final Schema schema = schemaFactory.newSchema(schemaUrl);
-        final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance();
-        docFactory.setSchema(schema);
-        final DocumentBuilder builder = docFactory.newDocumentBuilder();
-
-        builder.setErrorHandler(new org.xml.sax.ErrorHandler() {
-            @Override
-            public void fatalError(final SAXParseException err) throws SAXException {
-                logger.error("Config file line " + err.getLineNumber() + ", col " + err.getColumnNumber() + ", uri " + err.getSystemId() + " :message: " + err.getMessage());
-                if (logger.isDebugEnabled()) {
-                    logger.error("Error Stack Dump", err);
-                }
-                throw err;
-            }
-
-            @Override
-            public void error(final SAXParseException err) throws SAXParseException {
-                logger.error("Config file line " + err.getLineNumber() + ", col " + err.getColumnNumber() + ", uri " + err.getSystemId() + " :message: " + err.getMessage());
-                if (logger.isDebugEnabled()) {
-                    logger.error("Error Stack Dump", err);
-                }
-                throw err;
-            }
-
-            @Override
-            public void warning(final SAXParseException err) throws SAXParseException {
-                logger.warn(" Config file line " + err.getLineNumber() + ", uri " + err.getSystemId() + " : message : " + err.getMessage());
-                if (logger.isDebugEnabled()) {
-                    logger.warn("Warning stack dump", err);
-                }
-                throw err;
-            }
-        });
-
-        // build the docuemnt
-        final Document document = builder.parse(xmlFile);
-
-        // ensure schema compliance
-        final Validator validator = schema.newValidator();
-        validator.validate(new DOMSource(document));
-
-        return document;
-    }
-
-    private void addBulletin(final Node node, final Severity severity, final String msg) {
-        addBulletin(node.getNodeId(), severity, msg);
-    }
-
-    private void addBulletin(final NodeIdentifier nodeId, final Severity severity, final String msg) {
-        bulletinRepository.addBulletin(BulletinFactory.createBulletin(BULLETIN_CATEGORY, severity.toString(),
-                nodeId.getApiAddress() + ":" + nodeId.getApiPort() + " -- " + msg));
-    }
-
-    /**
-     * Services a disconnection request.
-     *
-     * @param nodeId a node identifier
-     * @param userDn the DN of the user requesting the disconnection
-     *
-     * @throws UnknownNodeException if the node does not exist
-     * @throws IllegalNodeDisconnectionException if the node cannot be
-     * disconnected due to the cluster's state (e.g., node is last connected
-     * node or node is primary)
-     * @throws NodeDisconnectionException if the disconnection message fails to
-     * be sent.
-     */
-    @Override
-    public void requestDisconnection(final String nodeId, final String userDn) throws UnknownNodeException, IllegalNodeDisconnectionException, NodeDisconnectionException {
-        writeLock.lock();
-        try {
-            // check that the node is known
-            final Node node = getNode(nodeId);
-            if (node == null) {
-                throw new UnknownNodeException("Node does not exist.");
-            }
-            requestDisconnection(node.getNodeId(), /* ignore last node */ false, "User " + userDn + " Disconnected Node");
-        } finally {
-            writeLock.unlock("requestDisconnection(String)");
-        }
-    }
-
-    /**
-     * Requests a disconnection to the node with the given node ID, but any
-     * exception thrown is suppressed.
-     *
-     * @param nodeId the node ID
-     */
-    private void requestDisconnectionQuietly(final NodeIdentifier nodeId, final String explanation) {
-        try {
-            requestDisconnection(nodeId, /* ignore node check */ true, explanation);
-        } catch (final IllegalNodeDisconnectionException | NodeDisconnectionException ex) { /* suppress exception */ }
-    }
-
-    /**
-     * Issues a disconnection message to the node identified by the given node
-     * ID. If the node is not known, then a UnknownNodeException is thrown. If
-     * the node cannot be disconnected due to the cluster's state and
-     * ignoreLastNodeCheck is false, then a IllegalNodeDisconnectionException is
-     * thrown. Otherwise, a disconnection message is issued to the node.
-     *
-     * Whether the disconnection message is successfully sent to the node, the
-     * node is marked as disconnected and if the node is the primary node, then
-     * the primary role is revoked.
-     *
-     * @param nodeId the ID of the node
-     * @param ignoreNodeChecks if false, checks will be made to ensure the
-     * cluster supports the node's disconnection (e.g., the node is not the last
-     * connected node in the cluster; the node is not the primary); otherwise,
-     * the request is made regardless of the cluster state
-     * @param explanation
-     *
-     * @throws IllegalNodeDisconnectionException if the node cannot be
-     * disconnected due to the cluster's state (e.g., node is last connected
-     * node or node is primary). Not thrown if ignoreNodeChecks is true.
-     * @throws NodeDisconnectionException if the disconnection message fails to
-     * be sent.
-     */
-    private void requestDisconnection(final NodeIdentifier nodeId, final boolean ignoreNodeChecks, final String explanation)
-            throws IllegalNodeDisconnectionException, NodeDisconnectionException {
-
-        writeLock.lock();
-        try {
-
-            // check that the node is known
-            final Node node = getRawNode(nodeId.getId());
-            if (node == null) {
-                if (ignoreNodeChecks) {
-                    // issue the disconnection
-                    final DisconnectMessage request = new DisconnectMessage();
-                    request.setNodeId(nodeId);
-                    request.setExplanation(explanation);
-
-                    addEvent(nodeId, "Disconnection requested due to " + explanation);
-                    senderListener.disconnect(request);
-                    addEvent(nodeId, "Node disconnected due to " + explanation);
-                    addBulletin(nodeId, Severity.INFO, "Node disconnected due to " + explanation);
-                    return;
-                } else {
-                    throw new UnknownNodeException("Node does not exist");
-                }
-            }
-
-            // if necessary, check that the node may be disconnected
-            if (!ignoreNodeChecks) {
-                final Set<NodeIdentifier> connectedNodes = getNodeIds(Status.CONNECTED);
-                // cannot disconnect the last connected node in the cluster
-                if (connectedNodes.size() == 1 && connectedNodes.iterator().next().equals(nodeId)) {
-                    throw new IllegalNodeDisconnectionException("Node may not be disconnected because it is the only connected node in the cluster.");
-                } else if (isPrimaryNode(nodeId)) {
-                    // cannot disconnect the primary node in the cluster
-                    throw new IllegalNodeDisconnectionException("Node may not be disconnected because it is the primary node in the cluster.");
-                }
-            }
-
-            // update status
-            node.setStatus(Status.DISCONNECTED);
-            notifyDataFlowManagementServiceOfNodeStatusChange();
-
-            // issue the disconnection
-            final DisconnectMessage request = new DisconnectMessage();
-            request.setNodeId(nodeId);
-            request.setExplanation(explanation);
-
-            addEvent(nodeId, "Disconnection requested due to " + explanation);
-            senderListener.disconnect(request);
-            addEvent(nodeId, "Node disconnected due to " + explanation);
-            addBulletin(node, Severity.INFO, "Node disconnected due to " + explanation);
-        } finally {
-            writeLock.unlock("requestDisconnection(NodeIdentifier, boolean)");
-        }
-    }
-
-    /**
-     * Messages the node to have the primary role. If the messaging fails, then
-     * the node is marked as disconnected.
-     *
-     * @param nodeId the node ID to assign primary role
-     *
-     * @return true if primary role assigned; false otherwise
-     */
-    private boolean assignPrimaryRole(final NodeIdentifier nodeId) {
-        writeLock.lock();
-        try {
-            // create primary role message
-            final PrimaryRoleAssignmentMessage msg = new PrimaryRoleAssignmentMessage();
-            msg.setNodeId(nodeId);
-            msg.setPrimary(true);
-            logger.info("Attempting to assign primary role to node: " + nodeId);
-
-            // message 
-            senderListener.assignPrimaryRole(msg);
-
-            logger.info("Assigned primary role to node: " + nodeId);
-            addBulletin(nodeId, Severity.INFO, "Node assigned primary role");
-
-            // true indicates primary role assigned
-            return true;
-
-        } catch (final ProtocolException ex) {
-
-            logger.warn("Failed attempt to assign primary role to node " + nodeId + " due to " + ex);
-            addBulletin(nodeId, Severity.ERROR, "Failed to assign primary role to node due to: " + ex);
-
-            // mark node as disconnected and log/record the event
-            final Node node = getRawNode(nodeId.getId());
-            node.setStatus(Status.DISCONNECTED);
-            addEvent(node.getNodeId(), "Disconnected because of failed attempt to assign primary role.");
-
-            addBulletin(nodeId, Severity.WARNING, "Node disconnected because of failed attempt to assign primary role");
-
-            // false indicates primary role failed to be assigned
-            return false;
-        } finally {
-            writeLock.unlock("assignPrimaryRole");
-        }
-    }
-
-    /**
-     * Messages the node with the given node ID to no longer have the primary
-     * role. If the messaging fails, then the node is marked as disconnected.
-     *
-     * @return true if the primary role was revoked from the node; false
-     * otherwise
-     */
-    private boolean revokePrimaryRole(final NodeIdentifier nodeId) {
-        writeLock.lock();
-        try {
-            // create primary role message
-            final PrimaryRoleAssignmentMessage msg = new PrimaryRoleAssignmentMessage();
-            msg.setNodeId(nodeId);
-            msg.setPrimary(false);
-            logger.info("Attempting to revoke primary role from node: " + nodeId);
-
-            // send message
-            senderListener.assignPrimaryRole(msg);
-
-            logger.info("Revoked primary role from node: " + nodeId);
-            addBulletin(nodeId, Severity.INFO, "Primary Role revoked from node");
-
-            // true indicates primary role was revoked
-            return true;
-        } catch (final ProtocolException ex) {
-
-            logger.warn("Failed attempt to revoke primary role from node " + nodeId + " due to " + ex);
-
-            // mark node as disconnected and log/record the event
-            final Node node = getRawNode(nodeId.getId());
-            node.setStatus(Status.DISCONNECTED);
-            addEvent(node.getNodeId(), "Disconnected because of failed attempt to revoke primary role.");
-            addBulletin(node, Severity.ERROR, "Node disconnected because of failed attempt to revoke primary role");
-
-            // false indicates primary role failed to be revoked
-            return false;
-        } finally {
-            writeLock.unlock("revokePrimaryRole");
-        }
-    }
-
-    private NodeIdentifier addRequestorDn(final NodeIdentifier nodeId, final String dn) {
-        return new NodeIdentifier(nodeId.getId(), nodeId.getApiAddress(),
-                nodeId.getApiPort(), nodeId.getSocketAddress(), nodeId.getSocketPort(), dn);
-    }
-
-    private ConnectionResponseMessage handleConnectionRequest(final ConnectionRequestMessage requestMessage) {
-        final NodeIdentifier proposedIdentifier = requestMessage.getConnectionRequest().getProposedNodeIdentifier();
-        final ConnectionRequest requestWithDn = new ConnectionRequest(addRequestorDn(proposedIdentifier, requestMessage.getRequestorDN()));
-
-        final ConnectionResponse response = requestConnection(requestWithDn);
-        final ConnectionResponseMessage responseMessage = new ConnectionResponseMessage();
-        responseMessage.setConnectionResponse(response);
-        return responseMessage;
-    }
-
-    private void handleControllerStartupFailure(final ControllerStartupFailureMessage msg) {
-        writeLock.lock();
-        try {
-            final Node node = getRawNode(msg.getNodeId().getId());
-            if (node != null) {
-                node.setStatus(Status.DISCONNECTED);
-                addEvent(msg.getNodeId(), "Node could not join cluster because it failed to start up properly. Setting node to Disconnected. Node reported the following error: " + msg.getExceptionMessage());
-                addBulletin(node, Severity.ERROR, "Node could not join cluster because it failed to start up properly. Setting node to Disconnected. Node reported the following error: " + msg.getExceptionMessage());
-            }
-        } finally {
-            writeLock.unlock("handleControllerStartupFailure");
-        }
-    }
-
-    private void handleReconnectionFailure(final ReconnectionFailureMessage msg) {
-        writeLock.lock();
-        try {
-            final Node node = getRawNode(msg.getNodeId().getId());
-            if (node != null) {
-                node.setStatus(Status.DISCONNECTED);
-                final String errorMsg = "Node could not rejoin cluster. Setting node to Disconnected. Node reported the following error: " + msg.getExceptionMessage();
-                addEvent(msg.getNodeId(), errorMsg);
-                addBulletin(node, Severity.ERROR, errorMsg);
-            }
-        } finally {
-            writeLock.unlock("handleControllerStartupFailure");
-        }
-    }
-
-    /**
-     * Adds an instance of a specified controller service.
-     *
-     * @param type
-     * @param id
-     * @param properties
-     * @return
-     */
-    @Override
-    public ControllerServiceNode createControllerService(String type, String id, Map<String, String> properties) {
-        return controllerServiceProvider.createControllerService(type, id, properties);
-    }
-
-    @Override
-    public ControllerService getControllerService(String serviceIdentifier) {
-        return controllerServiceProvider.getControllerService(serviceIdentifier);
-    }
-
-    @Override
-    public ControllerServiceNode getControllerServiceNode(final String id) {
-        return controllerServiceProvider.getControllerServiceNode(id);
-    }
-
-    @Override
-    public boolean isControllerServiceEnabled(final ControllerService service) {
-        return controllerServiceProvider.isControllerServiceEnabled(service);
-    }
-
-    @Override
-    public boolean isControllerServiceEnabled(final String serviceIdentifier) {
-        return controllerServiceProvider.isControllerServiceEnabled(serviceIdentifier);
-    }
-
-    /**
-     * Handle a bulletins message.
-     *
-     * @param bulletins
-     */
-    public void handleBulletins(final NodeBulletins bulletins) {
-        final NodeIdentifier nodeIdentifier = bulletins.getNodeIdentifier();
-        final String nodeAddress = nodeIdentifier.getApiAddress() + ":" + nodeIdentifier.getApiPort();
-
-        // unmarshal the message
-        BulletinsPayload payload = BulletinsPayload.unmarshal(bulletins.getPayload());
-        for (final Bulletin bulletin : payload.getBulletins()) {
-            bulletin.setNodeAddress(nodeAddress);
-            bulletinRepository.addBulletin(bulletin);
-        }
-    }
-
-    /**
-     * Handles a node's heartbeat. If this heartbeat is a node's first heartbeat
-     * since its connection request, then the manager will mark the node as
-     * connected. If the node was previously disconnected due to a lack of
-     * heartbeat, then a reconnection request is issued. If the node was
-     * disconnected for other reasons, then a disconnection request is issued.
-     * If this instance is configured with a firewall and the heartbeat is
-     * blocked, then a disconnection request is issued.
-     *
-     * @param heartbeat
-     */
-    @Override
-    public void handleHeartbeat(final Heartbeat heartbeat) {
-        // sanity check heartbeat
-        if (heartbeat == null) {
-            throw new IllegalArgumentException("Heartbeat may not be null.");
-        } else if (heartbeat.getNodeIdentifier() == null) {
-            throw new IllegalArgumentException("Heartbeat does not contain a node ID.");
-        }
-
-        /*
-         * Processing a heartbeat requires a write lock, which may take a while
-         * to obtain.  Only the last heartbeat is necessary to process per node.
-         * Futhermore, since many could pile up, heartbeats are processed in 
-         * bulk.
-         * 
-         * The below queue stores the pending heartbeats.
-         */
-        pendingHeartbeats.add(heartbeat);
-    }
-
-    private void processPendingHeartbeats() {
-        Node node;
-
-        writeLock.lock();
-        try {
-            /*
-             * Get the most recent heartbeats for the nodes in the cluster.  This
-             * is achieved by "draining" the pending heartbeats queue, populating
-             * a map that associates a node identifier with its latest heartbeat, and
-             * finally, getting the values of the map.
-             */
-            final Map<NodeIdentifier, Heartbeat> mostRecentHeartbeatsMap = new HashMap<>();
-            Heartbeat aHeartbeat;
-            while ((aHeartbeat = pendingHeartbeats.poll()) != null) {
-                mostRecentHeartbeatsMap.put(aHeartbeat.getNodeIdentifier(), aHeartbeat);
-            }
-            final Collection<Heartbeat> mostRecentHeartbeats = new ArrayList<>(mostRecentHeartbeatsMap.values());
-
-            // return fast if no work to do
-            if (mostRecentHeartbeats.isEmpty()) {
-                return;
-            }
-
-            logNodes("Before Heartbeat Processing", heartbeatLogger);
-
-            final int numPendingHeartbeats = mostRecentHeartbeats.size();
-            if (heartbeatLogger.isDebugEnabled()) {
-                heartbeatLogger.debug(String.format("Handling %s heartbeat%s", numPendingHeartbeats, (numPendingHeartbeats > 1) ? "s" : ""));
-            }
-
-            for (final Heartbeat mostRecentHeartbeat : mostRecentHeartbeats) {
-                try {
-                    // resolve the proposed node identifier to valid node identifier
-                    final NodeIdentifier resolvedNodeIdentifier = resolveProposedNodeIdentifier(mostRecentHeartbeat.getNodeIdentifier());
-
-                    // get a raw reference to the node (if it doesn't exist, node will be null)
-                    node = getRawNode(resolvedNodeIdentifier.getId());
-
-                    // if the node thinks it has the primary role, but the manager has assigned the role to a different node, then revoke the role
-                    if (mostRecentHeartbeat.isPrimary() && !isPrimaryNode(resolvedNodeIdentifier)) {
-                        addEvent(resolvedNodeIdentifier, "Heartbeat indicates node is running as primary node.  Revoking primary role because primary role is assigned to a different node.");
-                        revokePrimaryRole(resolvedNodeIdentifier);
-                    }
-
-                    final boolean heartbeatIndicatesNotYetConnected = !mostRecentHeartbeat.isConnected();
-
-                    if (isBlockedByFirewall(resolvedNodeIdentifier.getSocketAddress())) {
-                        if (node == null) {
-                            logger.info("Firewall blocked heartbeat received from unknown node " + resolvedNodeIdentifier + ".  Issuing disconnection request.");
-                        } else {
-                            // record event
-                            addEvent(resolvedNodeIdentifier, "Firewall blocked received heartbeat.  Issuing disconnection request.");
-                        }
-
-                        // request node to disconnect
-                        requestDisconnectionQuietly(resolvedNodeIdentifier, "Blocked By Firewall");
-
-                    } else if (node == null) {  // unknown node, so issue reconnect request
-                        // create new node and add to node set
-                        final Node newNode = new Node(resolvedNodeIdentifier, Status.DISCONNECTED);
-                        nodes.add(newNode);
-
-                        // record event
-                        addEvent(newNode.getNodeId(), "Received heartbeat from unknown node.  Issuing reconnection request.");
-
-                        // record heartbeat
-                        newNode.setHeartbeat(mostRecentHeartbeat);
-                        requestReconnection(resolvedNodeIdentifier.getId(), "NCM Heartbeat Processing");
-                    } else if (heartbeatIndicatesNotYetConnected) {
-                        if (Status.CONNECTED == node.getStatus()) {
-                            // record event
-                            addEvent(node.getNodeId(), "Received heartbeat from node that thinks it is not yet part of the cluster, though the Manager thought it was. Marking as Disconnected and issuing reconnection request.");
-
-                            // record heartbeat
-                            node.setHeartbeat(null);
-                            node.setStatus(Status.DISCONNECTED);
-
-                            requestReconnection(resolvedNodeIdentifier.getId(), "NCM Heartbeat Processing");
-                        }
-                    } else if (Status.DISCONNECTED == node.getStatus()) {
-                        // ignore heartbeats from nodes disconnected by means other than lack of heartbeat, unless it is
-                        // the only node. We allow it if it is the only node because if we have a one-node cluster, then
-                        // we cannot manually reconnect it.
-                        if (node.isHeartbeatDisconnection() || nodes.size() == 1) {
-                            // record event
-                            if (node.isHeartbeatDisconnection()) {
-                                addEvent(resolvedNodeIdentifier, "Received heartbeat from node previously disconnected due to lack of heartbeat.  Issuing reconnection request.");
-                            } else {
-                                addEvent(resolvedNodeIdentifier, "Received heartbeat from node previously disconnected, but it is the only known node, so issuing reconnection request.");
-                            }
-
-                            // record heartbeat
-                            node.setHeartbeat(mostRecentHeartbeat);
-
-                            // request reconnection
-                            requestReconnection(resolvedNodeIdentifier.getId(), "NCM Heartbeat Processing");
-                        } else {
-                            // disconnected nodes should not heartbeat, so we need to issue a disconnection request
-                            heartbeatLogger.info("Ignoring received heartbeat from disconnected node " + resolvedNodeIdentifier + ".  Issuing disconnection request.");
-
-                            // request node to disconnect
-                            requestDisconnectionQuietly(resolvedNodeIdentifier, "Received Heartbeat from Node, but Manager has already marked Node as Disconnected");
-                        }
-
-                    } else if (Status.DISCONNECTING == node.getStatus()) {
-                        /* ignore spurious heartbeat */
-                    } else {  // node is either either connected or connecting
-                        // first heartbeat causes status change from connecting to connected
-                        if (Status.CONNECTING == node.getStatus()) {
-                            if (mostRecentHeartbeat.getCreatedTimestamp() < node.getConnectionRequestedTimestamp()) {
-                                heartbeatLogger.info("Received heartbeat for node " + resolvedNodeIdentifier + " but ignoring because it was generated before the node was last asked to reconnect.");
-                                continue;
-                            }
-
-                            // set status to connected
-                            node.setStatus(Status.CONNECTED);
-
-                            // record event
-                            addEvent(resolvedNodeIdentifier, "Received first heartbeat from connecting node.  Setting node to connected.");
-
-                            // notify service of updated node set
-                            notifyDataFlowManagementServiceOfNodeStatusChange();
-
-                            addBulletin(node, Severity.INFO, "Node Connected");
-                        } else {
-                            heartbeatLogger.info("Received heartbeat for node " + resolvedNodeIdentifier + ".");
-                        }
-
-                        // record heartbeat
-                        node.setHeartbeat(mostRecentHeartbeat);
-
-                        ComponentStatusRepository statusRepository = componentMetricsRepositoryMap.get(node.getNodeId());
-                        if (statusRepository == null) {
-                            statusRepository = createComponentStatusRepository();
-                            componentMetricsRepositoryMap.put(node.getNodeId(), statusRepository);
-                        }
-
-                        // If it's been a while since we've captured, capture this metric.
-                        final Date lastCaptureDate = statusRepository.getLastCaptureDate();
-                        final long millisSinceLastCapture = (lastCaptureDate == null) ? Long.MAX_VALUE : (System.currentTimeMillis() - lastCaptureDate.getTime());
-
-                        if (millisSinceLastCapture > componentStatusSnapshotMillis) {
-                            statusRepository.capture(node.getHeartbeatPayload().getProcessGroupStatus());
-                        }
-                    }
-                } catch (final Exception e) {
-                    logger.error("Failed to process heartbeat from {}:{} due to {}", mostRecentHeartbeat.getNodeIdentifier().getApiAddress(), mostRecentHeartbeat.getNodeIdentifier().getApiPort(), e.toString());
-                    if (logger.isDebugEnabled()) {
-                        logger.error("", e);
-                    }
-                }
-            }
-
-            logNodes("After Heartbeat Processing", heartbeatLogger);
-        } finally {
-            writeLock.unlock("processPendingHeartbeats");
-        }
-    }
-
-    private ComponentStatusRepository createComponentStatusRepository() {
-        final String implementationClassName = properties.getProperty(NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION, DEFAULT_COMPONENT_STATUS_REPO_IMPLEMENTATION);
-        if (implementationClassName == null) {
-            throw new RuntimeException("Cannot create Component Status Repository because the NiFi Properties is missing the following property: "
-                    + NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION);
-        }
-
-        try {
-            return NarThreadContextClassLoader.createInstance(implementationClassName, ComponentStatusRepository.class);
-        } catch (final Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    @Override
-    public Set<Node> getNodes(final Status... statuses) {
-        final Set<Status> desiredStatusSet = new HashSet<>();
-        for (final Status status : statuses) {
-            desiredStatusSet.add(status);
-        }
-
-        readLock.lock();
-        try {
-            final Set<Node> clonedNodes = new HashSet<>();
-            for (final Node node : nodes) {
-                if (desiredStatusSet.isEmpty() || desiredStatusSet.contains(node.getStatus())) {
-                    clonedNodes.add(node.clone());
-                }
-            }
-            return Collections.unmodifiableSet(clonedNodes);
-        } finally {
-            readLock.unlock("getNodes(Status...)");
-        }
-    }
-
-    @Override
-    public Node getNode(final String nodeId) {
-        readLock.lock();
-        try {
-            for (final Node node : nodes) {
-                if (node.getNodeId().getId().equals(nodeId)) {
-                    return node.clone();
-                }
-            }
-            return null;
-        } finally {
-            readLock.unlock("getNode(String)");
-        }
-    }
-
-    @Override
-    public Node getPrimaryNode() {
-        readLock.lock();
-        try {
-            if (primaryNodeId == null) {
-                return null;
-            } else {
-                return getNode(primaryNodeId.getId());
-            }
-        } finally {
-            readLock.unlock("getPrimaryNode");
-        }
-    }
-
-    @Override
-    public void deleteNode(final String nodeId, final String userDn) throws UnknownNodeException, IllegalNodeDeletionException {
-        writeLock.lock();
-        try {
-            final Node node = getNode(nodeId);
-            if (node == null) {
-                throw new UnknownNodeException("Node does not exist.");
-            } else if (Status.DISCONNECTED == node.getStatus()) {
-                nodes.remove(node);
-
-                if (eventManager != null) {
-                    eventManager.clearEventHistory(node.getNodeId().getId());
-                }
-
-                logger.info("Removing node {} from cluster because this action was requested by {}", node, userDn);
-            } else {
-                throw new IllegalNodeDeletionException("Node may not be deleted because it is not disconnected.");
-            }
-        } finally {
-            writeLock.unlock("deleteNode");
-        }
-    }
-
-    @Override
-    public Set<NodeIdentifier> getNodeIds(final Status... statuses) {
-        readLock.lock();
-        try {
-            final Set<NodeIdentifier> nodeIds = new HashSet<>();
-            for (final Node node : nodes) {
-                if (statuses == null || statuses.length == 0) {
-                    nodeIds.add(node.getNodeId());
-                } else {
-                    for (final Node.Status status : statuses) {
-                        if (node.getStatus() == status) {
-                            nodeIds.add(node.getNodeId());
-                            break;
-                        }
-                    }
-                }
-            }
-            return nodeIds;
-        } finally {
-            readLock.unlock("getNodeIds(Status...)");
-        }
-    }
-
-    @Override
-    public void setPrimaryNode(final String nodeId, final String userDn) throws UnknownNodeException, IneligiblePrimaryNodeException, PrimaryRoleAssignmentException {
-        writeLock.lock();
-        try {
-
-            final Node node = getNode(nodeId);
-            if (node == null) {
-                throw new UnknownNodeException("Node does not exist.");
-            } else if (Status.CONNECTED != node.getStatus()) {
-                throw new IneligiblePrimaryNodeException("Node must be connected before it can be assigned as the primary node.");
-            }
-
-            // revoke primary role
-            final Node primaryNode;
-            if ((primaryNode = getPrimaryNode()) != null) {
-                if (primaryNode.getStatus() == Status.DISCONNECTED) {
-                    throw new PrimaryRoleAssignmentException("A disconnected, primary node exists.  Delete the node before assigning the primary role to a different node.");
-                } else if (revokePrimaryRole(primaryNode.getNodeId())) {
-                    addEvent(primaryNode.getNodeId(), "Role revoked from this node as part of primary role reassignment.");
-                } else {
-                    throw new PrimaryRoleAssignmentException(
-                            "Failed to revoke primary role from node. Primary node is now disconnected. Delete the node before assigning the primary role to a different node.");
-                }
-            }
-
-            // change the primary node ID to the given node
-            setPrimaryNodeId(node.getNodeId());
-
-            // assign primary role
-            if (assignPrimaryRole(node.getNodeId())) {
-                addEvent(node.getNodeId(), "Role assigned to this node as part of primary role reassignment. Action performed by " + userDn);
-                addBulletin(node, Severity.INFO, "Primary Role assigned to node by " + userDn);
-            } else {
-                throw new PrimaryRoleAssignmentException(
-                        "Cluster manager assigned primary role to node, but the node failed to accept the assignment.  Cluster manager disconnected node.");
-            }
-        } finally {
-            writeLock.unlock("setPrimaryNode");
-        }
-    }
-
-    private int getClusterProtocolHeartbeatSeconds() {
-        return (int) FormatUtils.getTimeDuration(properties.getClusterProtocolHeartbeatInterval(), TimeUnit.SECONDS);
-    }
-
-    @Override
-    public int getHeartbeatMonitoringIntervalSeconds() {
-        return 4 * getClusterProtocolHeartbeatSeconds();
-    }
-
-    @Override
-    public int getMaxHeartbeatGapSeconds() {
-        return 8 * getClusterProtocolHeartbeatSeconds();
-    }
-
-    @Override
-    public List<Event> getNodeEvents(final String nodeId) {
-        readLock.lock();
-        try {
-            List<Event> events = null;
-            final EventManager eventMgr = eventManager;
-            if (eventMgr != null) {
-                events = eventMgr.getEvents(nodeId);
-            }
-
-            if (events == null) {
-                return Collections.emptyList();
-            } else {
-                return Collections.unmodifiableList(events);
-            }
-        } finally {
-            readLock.unlock("getNodeEvents");
-        }
-    }
-
-    @Override
-    public NodeResponse applyRequest(final String method, final URI uri, final Map<String, List<String>> parameters, final Map<String, String> headers)
-            throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException, DisconnectedNodeMutableRequestException, SafeModeMutableRequestException {
-        return applyRequest(method, uri, parameters, headers, getNodeIds(Status.CONNECTED));
-    }
-
-    @Override
-    public NodeResponse applyRequest(final String method, final URI uri, final Map<String, List<String>> parameters, final Map<String, String> headers, final Set<NodeIdentifier> nodeIdentifiers)
-            throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException, DisconnectedNodeMutableRequestException, SafeModeMutableRequestException {
-
-        final boolean mutableRequest = canChangeNodeState(method, uri);
-        final ClusterManagerLock lock = mutableRequest ? writeLock : readLock;
-
-        lock.lock();
-        try {
-            // check that the request can be applied
-            if (mutableRequest) {
-                if (isInSafeMode()) {
-                    throw new SafeModeMutableRequestException("Received a mutable request [" + method + " -- " + uri + "] while in safe mode");
-                } else if (!getNodeIds(Status.DISCONNECTED, Status.DISCONNECTING).isEmpty()) {
-                    throw new DisconnectedNodeMutableRequestException("Received a mutable request [" + method + " -- " + uri + "] while a node is disconnected from the cluster");
-                } else if (!getNodeIds(Status.CONNECTING).isEmpty()) {
-                    // if any node is connecting and a request can change the flow, then we throw an exception
-                    throw new ConnectingNodeMutableRequestException("Received a mutable request [" + method + " -- " + uri + "] while a node is trying to connect to the cluster");
-                }
-            }
-
-            final NodeResponse clientResponse = federateRequest(method, uri, parameters, null, headers, nodeIdentifiers);
-            if (clientResponse == null) {
-                if (mutableRequest) {
-                    throw new NoConnectedNodesException(String.format("All nodes were disconnected as a result of applying request %s %s", method, uri));
-                } else {
-                    throw new NoResponseFromNodesException("No nodes were able to process this request.");
-                }
-            } else {
-                return clientResponse;
-            }
-        } finally {
-            lock.unlock("applyRequest(String, URI, Map<String, List<String>>, Map<String, String>, Set<NodeIdentifier>");
-        }
-    }
-
-    @Override
-    public NodeResponse applyRequest(final String method, final URI uri, final Object entity, final Map<String, String> headers)
-            throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException, DisconnectedNodeMutableRequestException, SafeModeMutableRequestException {
-        return applyRequest(method, uri, entity, headers, getNodeIds(Status.CONNECTED));
-    }
-
-    @Override

<TRUNCATED>

[40/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastUtils.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastUtils.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastUtils.java
deleted file mode 100644
index 8a8b7c0..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastUtils.java
+++ /dev/null
@@ -1,109 +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.io.socket.multicast;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.MulticastSocket;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author unattributed
- */
-public final class MulticastUtils {
-
-    private static final Logger logger = new org.apache.nifi.logging.NiFiLog(LoggerFactory.getLogger(MulticastUtils.class));
-
-    public static MulticastSocket createMulticastSocket(final MulticastConfiguration config) throws IOException {
-        return createMulticastSocket(0, config);
-    }
-
-    public static MulticastSocket createMulticastSocket(final int port, final MulticastConfiguration config) throws IOException {
-        if (config == null) {
-            throw new IllegalArgumentException("Configuration may not be null.");
-        }
-
-        final MulticastSocket socket;
-        if (port <= 0) {
-            socket = new MulticastSocket();
-        } else {
-            socket = new MulticastSocket(port);
-        }
-        socket.setTimeToLive(config.getTtl().getTtl());
-
-        if (config.getSocketTimeout() != null) {
-            socket.setSoTimeout(config.getSocketTimeout());
-        }
-
-        if (config.getReuseAddress() != null) {
-            socket.setReuseAddress(config.getReuseAddress());
-        }
-
-        if (config.getReceiveBufferSize() != null) {
-            socket.setReceiveBufferSize(config.getReceiveBufferSize());
-        }
-
-        if (config.getSendBufferSize() != null) {
-            socket.setSendBufferSize(config.getSendBufferSize());
-        }
-
-        if (config.getTrafficClass() != null) {
-            socket.setTrafficClass(config.getTrafficClass());
-        }
-
-        if (config.getLoopbackMode() != null) {
-            socket.setLoopbackMode(config.getLoopbackMode());
-        }
-
-        return socket;
-    }
-
-    public static void closeQuietly(final MulticastSocket socket) {
-
-        if (socket == null) {
-            return;
-        }
-
-        try {
-            socket.close();
-        } catch (final Exception ex) {
-            logger.debug("Failed to close multicast socket due to: " + ex, ex);
-        }
-
-    }
-
-    public static void closeQuietly(final MulticastSocket socket, final InetAddress groupAddress) {
-
-        if (socket == null) {
-            return;
-        }
-
-        try {
-            socket.leaveGroup(groupAddress);
-        } catch (final Exception ex) {
-            logger.debug("Failed to leave multicast group due to: " + ex, ex);
-        }
-
-        try {
-            socket.close();
-        } catch (final Exception ex) {
-            logger.debug("Failed to close multicast socket due to: " + ex, ex);
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/ServiceDiscovery.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/ServiceDiscovery.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/ServiceDiscovery.java
deleted file mode 100644
index 173146e..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/ServiceDiscovery.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.io.socket.multicast;
-
-/**
- * Defines a generic interface for discovering services.
- *
- * @author unattributed
- */
-public interface ServiceDiscovery {
-
-    /**
-     * @return the discovered service
-     */
-    DiscoverableService getService();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/ServicesBroadcaster.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/ServicesBroadcaster.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/ServicesBroadcaster.java
deleted file mode 100644
index 86260d8..0000000
--- a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/ServicesBroadcaster.java
+++ /dev/null
@@ -1,56 +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.io.socket.multicast;
-
-import java.util.Set;
-
-/**
- * Defines the interface for broadcasting a collection of services for client
- * discovery.
- *
- * @author unattributed
- */
-public interface ServicesBroadcaster {
-
-    /**
-     * @return the delay in milliseconds to wait between successive broadcasts
-     */
-    int getBroadcastDelayMs();
-
-    /**
-     * @return the broadcasted services
-     */
-    Set<DiscoverableService> getServices();
-
-    /**
-     * Adds the given service to the set of broadcasted services.
-     *
-     * @param service a service
-     * @return true if the service was added to the set; false a service with
-     * the given service name already exists in the set.
-     */
-    boolean addService(DiscoverableService service);
-
-    /**
-     * Removes the service with the given service name from the set.
-     *
-     * @param serviceName a service name
-     * @return true if the service was removed; false otherwise
-     */
-    boolean removeService(String serviceName);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/ServerMain.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/ServerMain.java b/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/ServerMain.java
deleted file mode 100644
index b5240c9..0000000
--- a/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/ServerMain.java
+++ /dev/null
@@ -1,141 +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.io.nio.example;
-
-import java.io.IOException;
-import java.util.Calendar;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.io.nio.BufferPool;
-import org.apache.nifi.io.nio.ChannelListener;
-import org.apache.nifi.io.nio.consumer.StreamConsumer;
-import org.apache.nifi.io.nio.consumer.StreamConsumerFactory;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- *
- * @author none
- */
-public final class ServerMain {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(ServerMain.class);
-
-    public static void main(final String[] args) throws IOException {
-        System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
-        System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug");
-
-        final ScheduledExecutorService executor = Executors.newScheduledThreadPool(10);
-        final Map<StreamConsumer, ScheduledFuture<?>> consumerMap = new ConcurrentHashMap<>();
-        final BufferPool bufferPool = new BufferPool(10, 5 << 20, false, 40.0);
-        ChannelListener listener = null;
-        try {
-            executor.scheduleWithFixedDelay(bufferPool, 0L, 5L, TimeUnit.SECONDS);
-            listener = new ChannelListener(5, new ExampleStreamConsumerFactory(executor, consumerMap), bufferPool, 5, TimeUnit.MILLISECONDS);
-            listener.setChannelReaderSchedulingPeriod(50L, TimeUnit.MILLISECONDS);
-            listener.addDatagramChannel(null, 20000, 32 << 20);
-            LOGGER.info("Listening for UDP data on port 20000");
-            listener.addServerSocket(null, 20001, 64 << 20);
-            LOGGER.info("listening for TCP connections on port 20001");
-            listener.addServerSocket(null, 20002, 64 << 20);
-            LOGGER.info("listening for TCP connections on port 20002");
-            final Calendar endTime = Calendar.getInstance();
-            endTime.add(Calendar.MINUTE, 30);
-            while (true) {
-                processAllConsumers(consumerMap);
-                if (endTime.before(Calendar.getInstance())) {
-                    break; // time to shut down
-                }
-            }
-        } finally {
-            if (listener != null) {
-                LOGGER.info("Shutting down server....");
-                listener.shutdown(1L, TimeUnit.SECONDS);
-                LOGGER.info("Consumer map size = " + consumerMap.size());
-                while (consumerMap.size() > 0) {
-                    processAllConsumers(consumerMap);
-                }
-                LOGGER.info("Consumer map size = " + consumerMap.size());
-            }
-            executor.shutdown();
-        }
-    }
-
-    private static void processAllConsumers(final Map<StreamConsumer, ScheduledFuture<?>> consumerMap) {
-        final Set<StreamConsumer> deadConsumers = new HashSet<>();
-        for (final Map.Entry<StreamConsumer, ScheduledFuture<?>> entry : consumerMap.entrySet()) {
-            if (entry.getKey().isConsumerFinished()) {
-                entry.getValue().cancel(true);
-                deadConsumers.add(entry.getKey());
-            }
-        }
-        for (final StreamConsumer consumer : deadConsumers) {
-            LOGGER.debug("removing consumer " + consumer);
-            consumerMap.remove(consumer);
-        }
-    }
-
-    public static final class ConsumerRunner implements Runnable {
-
-        private final StreamConsumer consumer;
-
-        public ConsumerRunner(final StreamConsumer consumer) {
-            this.consumer = consumer;
-        }
-
-        @Override
-        public void run() {
-            if (consumer.isConsumerFinished()) {
-                return;
-            }
-            try {
-                consumer.process();
-            } catch (IOException ex) {
-                LOGGER.error("", ex);
-            }
-        }
-    }
-
-    public static final class ExampleStreamConsumerFactory implements StreamConsumerFactory {
-
-        final ScheduledExecutorService executor;
-        final Map<StreamConsumer, ScheduledFuture<?>> consumerMap;
-
-        public ExampleStreamConsumerFactory(final ScheduledExecutorService executor, final Map<StreamConsumer, ScheduledFuture<?>> consumerMap) {
-            this.executor = executor;
-            this.consumerMap = consumerMap;
-        }
-
-        @Override
-        public StreamConsumer newInstance(final String streamId) {
-            final StreamConsumer consumer = new UselessStreamConsumer(streamId);
-            final ScheduledFuture<?> future = executor.scheduleWithFixedDelay(new ConsumerRunner(consumer), 0L, 10L, TimeUnit.MILLISECONDS);
-            consumerMap.put(consumer, future);
-            LOGGER.info("Added consumer: " + consumer);
-            return consumer;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/TCPClient.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/TCPClient.java b/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/TCPClient.java
deleted file mode 100644
index b3d214e..0000000
--- a/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/TCPClient.java
+++ /dev/null
@@ -1,86 +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.io.nio.example;
-
-import java.io.IOException;
-import java.net.Socket;
-import java.net.SocketException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author none
- */
-public class TCPClient {
-
-    private static final Logger logger = LoggerFactory.getLogger(TCPClient.class);
-
-    public static void main(final String[] args) throws Exception {
-        final byte[] bytes = TCPClient.makeBytes();
-        Thread first = new Thread(new Runnable() {
-
-            @Override
-            public void run() {
-                try {
-                    for (int i = 0; i < 10; i++) {
-                        sendData(20001, bytes);
-                    }
-                } catch (Exception e) {
-                    logger.error("Blew exception", e);
-                }
-            }
-        });
-        Thread second = new Thread(new Runnable() {
-
-            @Override
-            public void run() {
-                try {
-                    for (int i = 0; i < 10; i++) {
-                        sendData(20002, bytes);
-                    }
-                } catch (Exception e) {
-                    logger.error("Blew exception", e);
-                }
-            }
-        });
-        first.start();
-        second.start();
-    }
-
-    public static byte[] makeBytes() {
-        byte[] bytes = new byte[2 << 20];
-        return bytes;
-    }
-
-    private static void sendData(final int port, final byte[] bytes) throws SocketException, IOException, InterruptedException {
-        long totalBytes;
-        try (Socket sock = new Socket("localhost", port)) {
-            sock.setTcpNoDelay(true);
-            sock.setSoTimeout(2000);
-            totalBytes = 0L;
-            logger.info("socket established " + sock + " to port " + port + " now waiting 5 seconds to send anything...");
-            Thread.sleep(5000L);
-            for (int i = 0; i < 1000; i++) {
-                sock.getOutputStream().write(bytes);
-                totalBytes += bytes.length;
-            }   sock.getOutputStream().flush();
-        }
-        logger.info("Total bytes sent: " + totalBytes + " to port " + port);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/UDPClient.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/UDPClient.java b/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/UDPClient.java
deleted file mode 100644
index 90f4c42..0000000
--- a/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/UDPClient.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.io.nio.example;
-
-import java.net.DatagramPacket;
-import java.net.DatagramSocket;
-import java.net.InetSocketAddress;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @author none
- */
-public class UDPClient {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(UDPClient.class);
-
-    public static void main(final String[] args) throws Exception {
-        final byte[] buffer = UDPClient.makeBytes();
-        final DatagramPacket packet = new DatagramPacket(buffer, buffer.length, new InetSocketAddress("localhost", 20000));
-        final DatagramSocket socket = new DatagramSocket();
-        final long startTime = System.nanoTime();
-        for (int i = 0; i < 819200; i++) { // 100 MB
-            socket.send(packet);
-        }
-        final long endTime = System.nanoTime();
-        final long durationMillis = (endTime - startTime) / 1000000;
-        LOGGER.info("Sent all UDP packets without any obvious errors | duration ms= " + durationMillis);
-    }
-
-    public static byte[] makeBytes() {
-        byte[] bytes = new byte[128];
-        return bytes;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/UselessStreamConsumer.java
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/UselessStreamConsumer.java b/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/UselessStreamConsumer.java
deleted file mode 100644
index 9ec26e9..0000000
--- a/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/UselessStreamConsumer.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.io.nio.example;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.nifi.io.nio.consumer.AbstractStreamConsumer;
-
-/**
- *
- * @author none
- */
-public class UselessStreamConsumer extends AbstractStreamConsumer {
-
-    public UselessStreamConsumer(final String id) {
-        super(id);
-    }
-
-    @Override
-    protected void processBuffer(final ByteBuffer buffer) throws IOException {
-    }
-
-    @Override
-    protected void onConsumerDone() {
-        System.err.println("IN consumer done");
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-socket-utils/src/test/resources/log4j.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/src/test/resources/log4j.xml b/commons/nifi-socket-utils/src/test/resources/log4j.xml
deleted file mode 100644
index 8e93769..0000000
--- a/commons/nifi-socket-utils/src/test/resources/log4j.xml
+++ /dev/null
@@ -1,36 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
-
-<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
-
-    <!-- Appender for printing formatted log statements to the console. -->
-    <appender name="console" class="org.apache.log4j.ConsoleAppender">
-        <layout class="org.apache.log4j.PatternLayout">
-            <param name="ConversionPattern" value="%d %-5p [%t] %40.40c - %m%n"/>
-        </layout>
-    </appender>
-
-    <!-- Logger for managing logging statements for nifi -->
-    <logger name="nifi">
-        <level value="debug"/>
-    </logger>
-
-    <root>
-        <level value="warn"/>
-        <appender-ref ref="console"/>
-    </root>
-</log4j:configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/.gitignore
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/.gitignore b/commons/nifi-utils/.gitignore
deleted file mode 100755
index 12c5231..0000000
--- a/commons/nifi-utils/.gitignore
+++ /dev/null
@@ -1,8 +0,0 @@
-/target
-/target
-/target
-/target
-/target
-/target
-/target
-/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/pom.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/pom.xml b/commons/nifi-utils/pom.xml
deleted file mode 100644
index c5c2a68..0000000
--- a/commons/nifi-utils/pom.xml
+++ /dev/null
@@ -1,33 +0,0 @@
-<?xml version="1.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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-commons-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-
-    <artifactId>nifi-utils</artifactId>
-    <version>0.0.1-SNAPSHOT</version>
-    <packaging>jar</packaging>
-    <name>NiFi Utils</name>
-    <!--
-    This project intentionally has no additional dependencies beyond that pulled in by the parent.  It is a general purpose utility library
-    and should keep its surface/tension minimal.
-    -->
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/flowfile/attributes/CoreAttributes.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/flowfile/attributes/CoreAttributes.java b/commons/nifi-utils/src/main/java/org/apache/nifi/flowfile/attributes/CoreAttributes.java
deleted file mode 100644
index 24f43ca..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/flowfile/attributes/CoreAttributes.java
+++ /dev/null
@@ -1,72 +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.flowfile.attributes;
-
-public enum CoreAttributes implements FlowFileAttributeKey {
-    /**
-     * The flowfile's path indicates the relative directory to which a FlowFile belongs and does not
-     * contain the filename
-     */
-    PATH("path"),
-    
-    /**
-     * The flowfile's absolute path indicates the absolute directory to which a FlowFile belongs and does not
-     * contain the filename
-     */
-    ABSOLUTE_PATH("absolute.path"),
-    
-    /**
-     * The filename of the FlowFile. The filename should not contain any directory structure.
-     */
-    FILENAME("filename"),
-    
-    /**
-     * A unique UUID assigned to this FlowFile
-     */
-    UUID("uuid"),
-    
-    /**
-     * A numeric value indicating the FlowFile priority
-     */
-    PRIORITY("priority"),
-    
-    /**
-     * The MIME Type of this FlowFile
-     */
-    MIME_TYPE("mime.type"),
-    
-    /**
-     * Specifies the reason that a FlowFile is being discarded
-     */
-    DISCARD_REASON("discard.reason"),
-
-    /**
-     * Indicates an identifier other than the FlowFile's UUID that is known to refer to this FlowFile.
-     */
-    ALTERNATE_IDENTIFIER("alternate.identifier");
-    
-    private final String key;
-    private CoreAttributes(final String key) {
-        this.key = key;
-    }
-    
-    @Override
-    public String key() {
-        return key;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/flowfile/attributes/FlowFileAttributeKey.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/flowfile/attributes/FlowFileAttributeKey.java b/commons/nifi-utils/src/main/java/org/apache/nifi/flowfile/attributes/FlowFileAttributeKey.java
deleted file mode 100644
index cc6c28e..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/flowfile/attributes/FlowFileAttributeKey.java
+++ /dev/null
@@ -1,21 +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.flowfile.attributes;
-
-public interface FlowFileAttributeKey {
-    String key();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/remote/StandardVersionNegotiator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/StandardVersionNegotiator.java b/commons/nifi-utils/src/main/java/org/apache/nifi/remote/StandardVersionNegotiator.java
deleted file mode 100644
index 77c34c9..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/StandardVersionNegotiator.java
+++ /dev/null
@@ -1,81 +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.remote;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Objects;
-
-public class StandardVersionNegotiator implements VersionNegotiator {
-
-    private final List<Integer> versions;
-    private int curVersion;
-
-    public StandardVersionNegotiator(final int... supportedVersions) {
-        if (Objects.requireNonNull(supportedVersions).length == 0) {
-            throw new IllegalArgumentException("At least one version must be supported");
-        }
-
-        final List<Integer> supported = new ArrayList<>();
-        for (final int version : supportedVersions) {
-            supported.add(version);
-        }
-        this.versions = Collections.unmodifiableList(supported);
-        this.curVersion = supportedVersions[0];
-    }
-
-    @Override
-    public int getVersion() {
-        return curVersion;
-    }
-
-    @Override
-    public void setVersion(final int version) throws IllegalArgumentException {
-        if (!isVersionSupported(version)) {
-            throw new IllegalArgumentException("Version " + version + " is not supported");
-        }
-
-        this.curVersion = version;
-    }
-
-    @Override
-    public int getPreferredVersion() {
-        return versions.get(0);
-    }
-
-    @Override
-    public Integer getPreferredVersion(final int maxVersion) {
-        for (final Integer version : this.versions) {
-            if (maxVersion >= version) {
-                return version;
-            }
-        }
-        return null;
-    }
-
-    @Override
-    public boolean isVersionSupported(final int version) {
-        return versions.contains(version);
-    }
-
-    @Override
-    public List<Integer> getSupportedVersions() {
-        return versions;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/remote/VersionNegotiator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/VersionNegotiator.java b/commons/nifi-utils/src/main/java/org/apache/nifi/remote/VersionNegotiator.java
deleted file mode 100644
index 74f9b3d..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/VersionNegotiator.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.remote;
-
-import java.util.List;
-
-public interface VersionNegotiator {
-
-    /**
-     * @return the currently configured Version of this resource
-     */
-    int getVersion();
-
-    /**
-     * Sets the version of this resource to the specified version. Only the
-     * lower byte of the version is relevant.
-     *
-     * @param version
-     * @throws IllegalArgumentException if the given Version is not supported by
-     * this resource, as is indicated by the {@link #isVersionSupported(int)}
-     * method
-     */
-    void setVersion(int version) throws IllegalArgumentException;
-
-    /**
-     *
-     * @return the Version of this resource that is preferred
-     */
-    int getPreferredVersion();
-
-    /**
-     * Gets the preferred version of this resource that is no greater than the
-     * given maxVersion. If no acceptable version exists that is less than
-     * <code>maxVersion</code>, then <code>null</code> is returned
-     *
-     * @param maxVersion
-     * @return
-     */
-    Integer getPreferredVersion(int maxVersion);
-
-    /**
-     * Indicates whether or not the specified version is supported by this
-     * resource
-     *
-     * @param version
-     * @return
-     */
-    boolean isVersionSupported(int version);
-
-    List<Integer> getSupportedVersions();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/remote/exception/TransmissionDisabledException.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/exception/TransmissionDisabledException.java b/commons/nifi-utils/src/main/java/org/apache/nifi/remote/exception/TransmissionDisabledException.java
deleted file mode 100644
index 05fd915..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/exception/TransmissionDisabledException.java
+++ /dev/null
@@ -1,25 +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.remote.exception;
-
-/**
- * Indicates that the user disabled transmission while communications were
- * taking place with a peer
- */
-public class TransmissionDisabledException extends RuntimeException {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java
deleted file mode 100644
index 71cf894..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java
+++ /dev/null
@@ -1,184 +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.remote.io;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Arrays;
-import java.util.zip.DataFormatException;
-import java.util.zip.Inflater;
-
-public class CompressionInputStream extends InputStream {
-
-    private final InputStream in;
-    private final Inflater inflater;
-
-    private byte[] compressedBuffer;
-    private byte[] buffer;
-
-    private int bufferIndex;
-    private boolean eos = false;    // whether or not we've reached the end of stream
-    private boolean allDataRead = false;    // different from eos b/c eos means allDataRead == true && buffer is empty
-
-    private final byte[] fourByteBuffer = new byte[4];
-
-    public CompressionInputStream(final InputStream in) {
-        this.in = in;
-        inflater = new Inflater();
-
-        buffer = new byte[0];
-        compressedBuffer = new byte[0];
-        bufferIndex = 1;
-    }
-
-    private String toHex(final byte[] array) {
-        final StringBuilder sb = new StringBuilder("0x");
-        for (final byte b : array) {
-            final String hex = Integer.toHexString(b).toUpperCase();
-            if (hex.length() == 1) {
-                sb.append("0");
-            }
-            sb.append(hex);
-        }
-        return sb.toString();
-    }
-
-    protected void readChunkHeader() throws IOException {
-        // Ensure that we have a valid SYNC chunk
-        fillBuffer(fourByteBuffer);
-        if (!Arrays.equals(CompressionOutputStream.SYNC_BYTES, fourByteBuffer)) {
-            throw new IOException("Invalid CompressionInputStream. Expected first 4 bytes to be 'SYNC' but were " + toHex(fourByteBuffer));
-        }
-
-        // determine the size of the decompressed buffer
-        fillBuffer(fourByteBuffer);
-        buffer = new byte[toInt(fourByteBuffer)];
-
-        // determine the size of the compressed buffer
-        fillBuffer(fourByteBuffer);
-        compressedBuffer = new byte[toInt(fourByteBuffer)];
-
-        bufferIndex = buffer.length;	// indicate that buffer is empty
-    }
-
-    private int toInt(final byte[] data) {
-        return ((data[0] & 0xFF) << 24)
-                | ((data[1] & 0xFF) << 16)
-                | ((data[2] & 0xFF) << 8)
-                | (data[3] & 0xFF);
-    }
-
-    protected void bufferAndDecompress() throws IOException {
-        if (allDataRead) {
-            eos = true;
-            return;
-        }
-
-        readChunkHeader();
-        fillBuffer(compressedBuffer);
-
-        inflater.setInput(compressedBuffer);
-        try {
-            inflater.inflate(buffer);
-        } catch (final DataFormatException e) {
-            throw new IOException(e);
-        }
-        inflater.reset();
-
-        bufferIndex = 0;
-        final int moreDataByte = in.read();
-        if (moreDataByte < 1) {
-            allDataRead = true;
-        } else if (moreDataByte > 1) {
-            throw new IOException("Expected indicator of whether or not more data was to come (-1, 0, or 1) but got " + moreDataByte);
-        }
-    }
-
-    private void fillBuffer(final byte[] buffer) throws IOException {
-        int len;
-        int bytesLeft = buffer.length;
-        int bytesRead = 0;
-        while (bytesLeft > 0 && (len = in.read(buffer, bytesRead, bytesLeft)) > 0) {
-            bytesLeft -= len;
-            bytesRead += len;
-        }
-
-        if (bytesRead < buffer.length) {
-            throw new EOFException();
-        }
-    }
-
-    private boolean isBufferEmpty() {
-        return bufferIndex >= buffer.length;
-    }
-
-    @Override
-    public int read() throws IOException {
-        if (eos) {
-            return -1;
-        }
-
-        if (isBufferEmpty()) {
-            bufferAndDecompress();
-        }
-
-        if (isBufferEmpty()) {
-            eos = true;
-            return -1;
-        }
-
-        return buffer[bufferIndex++];
-    }
-
-    @Override
-    public int read(final byte[] b) throws IOException {
-        return read(b, 0, b.length);
-    }
-
-    @Override
-    public int read(final byte[] b, final int off, final int len) throws IOException {
-        if (eos) {
-            return -1;
-        }
-
-        if (isBufferEmpty()) {
-            bufferAndDecompress();
-        }
-
-        if (isBufferEmpty()) {
-            eos = true;
-            return -1;
-        }
-
-        final int free = buffer.length - bufferIndex;
-        final int bytesToTransfer = Math.min(len, free);
-        System.arraycopy(buffer, bufferIndex, b, off, bytesToTransfer);
-        bufferIndex += bytesToTransfer;
-
-        return bytesToTransfer;
-    }
-
-    /**
-     * Does nothing. Does NOT close underlying InputStream
-     * @throws java.io.IOException
-     */
-    @Override
-    public void close() throws IOException {
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionOutputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionOutputStream.java
deleted file mode 100644
index bc46b0f..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionOutputStream.java
+++ /dev/null
@@ -1,147 +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.remote.io;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.zip.Deflater;
-
-public class CompressionOutputStream extends OutputStream {
-
-    public static final byte[] SYNC_BYTES = new byte[]{'S', 'Y', 'N', 'C'};
-
-    public static final int DEFAULT_COMPRESSION_LEVEL = 1;
-    public static final int DEFAULT_BUFFER_SIZE = 64 << 10;
-    public static final int MIN_BUFFER_SIZE = 8 << 10;
-
-    private final OutputStream out;
-    private final Deflater deflater;
-
-    private final byte[] buffer;
-    private final byte[] compressed;
-
-    private int bufferIndex = 0;
-    private boolean dataWritten = false;
-
-    public CompressionOutputStream(final OutputStream outStream) {
-        this(outStream, DEFAULT_BUFFER_SIZE);
-    }
-
-    public CompressionOutputStream(final OutputStream outStream, final int bufferSize) {
-        this(outStream, bufferSize, DEFAULT_COMPRESSION_LEVEL, Deflater.DEFAULT_STRATEGY);
-    }
-
-    public CompressionOutputStream(final OutputStream outStream, final int bufferSize, final int level, final int strategy) {
-        if (bufferSize < MIN_BUFFER_SIZE) {
-            throw new IllegalArgumentException("Buffer size must be at least " + MIN_BUFFER_SIZE);
-        }
-
-        this.out = outStream;
-        this.deflater = new Deflater(level);
-        this.deflater.setStrategy(strategy);
-        buffer = new byte[bufferSize];
-        compressed = new byte[bufferSize + 64];
-    }
-
-    /**
-     * Compresses the currently buffered chunk of data and sends it to the
-     * output stream
-     *
-     * @throws IOException
-     */
-    protected void compressAndWrite() throws IOException {
-        if (bufferIndex <= 0) {
-            return;
-        }
-
-        deflater.setInput(buffer, 0, bufferIndex);
-        deflater.finish();
-        final int compressedBytes = deflater.deflate(compressed);
-
-        writeChunkHeader(compressedBytes);
-        out.write(compressed, 0, compressedBytes);
-
-        bufferIndex = 0;
-        deflater.reset();
-    }
-
-    private void writeChunkHeader(final int compressedBytes) throws IOException {
-        // If we have already written data, write out a '1' to indicate that we have more data; when we close
-        // the stream, we instead write a '0' to indicate that we are finished sending data.
-        if (dataWritten) {
-            out.write(1);
-        }
-        out.write(SYNC_BYTES);
-        dataWritten = true;
-
-        writeInt(out, bufferIndex);
-        writeInt(out, compressedBytes);
-    }
-
-    private void writeInt(final OutputStream out, final int val) throws IOException {
-        out.write(val >>> 24);
-        out.write(val >>> 16);
-        out.write(val >>> 8);
-        out.write(val);
-    }
-
-    protected boolean bufferFull() {
-        return bufferIndex >= buffer.length;
-    }
-
-    @Override
-    public void write(final int b) throws IOException {
-        buffer[bufferIndex++] = (byte) (b & 0xFF);
-        if (bufferFull()) {
-            compressAndWrite();
-        }
-    }
-
-    @Override
-    public void write(final byte[] b) throws IOException {
-        write(b, 0, b.length);
-    }
-
-    @Override
-    public void write(final byte[] b, final int off, final int len) throws IOException {
-        int bytesLeft = len;
-        while (bytesLeft > 0) {
-            final int free = buffer.length - bufferIndex;
-            final int bytesThisIteration = Math.min(bytesLeft, free);
-            System.arraycopy(b, off + len - bytesLeft, buffer, bufferIndex, bytesThisIteration);
-            bufferIndex += bytesThisIteration;
-
-            bytesLeft -= bytesThisIteration;
-            if (bufferFull()) {
-                compressAndWrite();
-            }
-        }
-    }
-
-    @Override
-    public void flush() throws IOException {
-        compressAndWrite();
-        super.flush();
-    }
-
-    @Override
-    public void close() throws IOException {
-        compressAndWrite();
-        out.write(0);   // indicate that the stream is finished.
-        out.flush();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/InterruptableInputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/InterruptableInputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/InterruptableInputStream.java
deleted file mode 100644
index e03dfbf..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/InterruptableInputStream.java
+++ /dev/null
@@ -1,117 +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.remote.io;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.nifi.remote.exception.TransmissionDisabledException;
-
-public class InterruptableInputStream extends InputStream {
-
-    private volatile boolean interrupted = false;
-    private final InputStream in;
-
-    public InterruptableInputStream(final InputStream in) {
-        this.in = in;
-    }
-
-    @Override
-    public int read() throws IOException {
-        if (interrupted) {
-            throw new TransmissionDisabledException();
-        }
-
-        return in.read();
-    }
-
-    @Override
-    public int read(byte[] b) throws IOException {
-        if (interrupted) {
-            throw new TransmissionDisabledException();
-        }
-
-        return in.read(b);
-    }
-
-    @Override
-    public int read(byte[] b, int off, int len) throws IOException {
-        if (interrupted) {
-            throw new TransmissionDisabledException();
-        }
-
-        return in.read(b, off, len);
-    }
-
-    @Override
-    public int available() throws IOException {
-        if (interrupted) {
-            throw new TransmissionDisabledException();
-        }
-
-        return in.available();
-    }
-
-    @Override
-    public void close() throws IOException {
-        if (interrupted) {
-            throw new TransmissionDisabledException();
-        }
-
-        in.close();
-    }
-
-    @Override
-    public synchronized void mark(int readlimit) {
-        if (interrupted) {
-            throw new TransmissionDisabledException();
-        }
-
-        in.mark(readlimit);
-    }
-
-    @Override
-    public boolean markSupported() {
-        if (interrupted) {
-            throw new TransmissionDisabledException();
-        }
-
-        return in.markSupported();
-    }
-
-    @Override
-    public synchronized void reset() throws IOException {
-        if (interrupted) {
-            throw new TransmissionDisabledException();
-        }
-
-        in.reset();
-    }
-
-    @Override
-    public long skip(long n) throws IOException {
-        if (interrupted) {
-            throw new TransmissionDisabledException();
-        }
-
-        return in.skip(n);
-    }
-
-    public void interrupt() {
-        interrupted = true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/InterruptableOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/InterruptableOutputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/InterruptableOutputStream.java
deleted file mode 100644
index cba5be6..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/InterruptableOutputStream.java
+++ /dev/null
@@ -1,81 +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.remote.io;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-import org.apache.nifi.remote.exception.TransmissionDisabledException;
-
-public class InterruptableOutputStream extends OutputStream {
-
-    private final OutputStream out;
-    private volatile boolean interrupted = false;
-
-    public InterruptableOutputStream(final OutputStream out) {
-        this.out = out;
-    }
-
-    @Override
-    public void write(int b) throws IOException {
-        if (interrupted) {
-            throw new TransmissionDisabledException();
-        }
-
-        out.write(b);
-    }
-
-    @Override
-    public void write(byte[] b) throws IOException {
-        if (interrupted) {
-            throw new TransmissionDisabledException();
-        }
-
-        out.write(b);
-    }
-
-    @Override
-    public void write(byte[] b, int off, int len) throws IOException {
-        if (interrupted) {
-            throw new TransmissionDisabledException();
-        }
-
-        out.write(b, off, len);
-    }
-
-    @Override
-    public void close() throws IOException {
-        if (interrupted) {
-            throw new TransmissionDisabledException();
-        }
-
-        out.close();
-    }
-
-    @Override
-    public void flush() throws IOException {
-        if (interrupted) {
-            throw new TransmissionDisabledException();
-        }
-
-        out.flush();
-    }
-
-    public void interrupt() {
-        this.interrupted = true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/BufferStateManager.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/BufferStateManager.java b/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/BufferStateManager.java
deleted file mode 100644
index 68913bd..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/BufferStateManager.java
+++ /dev/null
@@ -1,111 +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.remote.io.socket;
-
-import java.nio.ByteBuffer;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class BufferStateManager {
-
-    private static final Logger logger = LoggerFactory.getLogger(BufferStateManager.class);
-
-    private ByteBuffer buffer;
-    private Direction direction = Direction.WRITE;
-
-    public BufferStateManager(final ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    public BufferStateManager(final ByteBuffer buffer, final Direction direction) {
-        this.buffer = buffer;
-        this.direction = direction;
-    }
-
-    /**
-     * Ensures that the buffer is at least as big as the size specified,
-     * resizing the buffer if necessary. This operation MAY change the direction
-     * of the buffer.
-     *
-     * @param requiredSize
-     */
-    public void ensureSize(final int requiredSize) {
-        if (buffer.capacity() < requiredSize) {
-            final ByteBuffer newBuffer = ByteBuffer.allocate(requiredSize);
-
-            // we have to read buffer so make sure the direction is correct.
-            if (direction == Direction.WRITE) {
-                buffer.flip();
-            }
-
-            // Copy from buffer to newBuffer
-            newBuffer.put(buffer);
-
-            // Swap the buffers
-            buffer = newBuffer;
-
-            // the new buffer is ready to be written to
-            direction = Direction.WRITE;
-        }
-    }
-
-    public ByteBuffer prepareForWrite(final int requiredSize) {
-        ensureSize(requiredSize);
-
-        if (direction == Direction.READ) {
-            direction = Direction.WRITE;
-            buffer.position(buffer.limit());
-        }
-
-        buffer.limit(buffer.capacity());
-        return buffer;
-    }
-
-    public ByteBuffer prepareForRead(final int requiredSize) {
-        ensureSize(requiredSize);
-
-        if (direction == Direction.WRITE) {
-            direction = Direction.READ;
-            buffer.flip();
-        }
-
-        return buffer;
-    }
-
-    /**
-     * Clears the contents of the buffer and sets direction to WRITE
-     */
-    public void clear() {
-        logger.debug("Clearing {}", buffer);
-        buffer.clear();
-        direction = Direction.WRITE;
-    }
-
-    public void compact() {
-        final String before = buffer.toString();
-        buffer.compact();
-        logger.debug("Before compact: {}, after: {}", before, buffer);
-        direction = Direction.WRITE;
-    }
-
-    public static enum Direction {
-
-        READ, WRITE;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java
deleted file mode 100644
index 32a3f26..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java
+++ /dev/null
@@ -1,157 +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.remote.io.socket;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.SocketTimeoutException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ClosedByInterruptException;
-import java.nio.channels.SocketChannel;
-import java.util.concurrent.TimeUnit;
-
-public class SocketChannelInputStream extends InputStream {
-
-    private static final long CHANNEL_EMPTY_WAIT_NANOS = TimeUnit.NANOSECONDS.convert(10, TimeUnit.MILLISECONDS);
-    private final SocketChannel channel;
-    private volatile int timeoutMillis = 30000;
-
-    private final ByteBuffer oneByteBuffer = ByteBuffer.allocate(1);
-    private Byte bufferedByte = null;
-
-    public SocketChannelInputStream(final SocketChannel socketChannel) throws IOException {
-        // this class expects a non-blocking channel
-        socketChannel.configureBlocking(false);
-        this.channel = socketChannel;
-    }
-
-    public void setTimeout(final int timeoutMillis) {
-        this.timeoutMillis = timeoutMillis;
-    }
-
-    @Override
-    public int read() throws IOException {
-        if (bufferedByte != null) {
-            final int retVal = bufferedByte & 0xFF;
-            bufferedByte = null;
-            return retVal;
-        }
-
-        oneByteBuffer.flip();
-        oneByteBuffer.clear();
-
-        final long maxTime = System.currentTimeMillis() + timeoutMillis;
-        int bytesRead;
-        do {
-            bytesRead = channel.read(oneByteBuffer);
-            if (bytesRead == 0) {
-                if (System.currentTimeMillis() > maxTime) {
-                    throw new SocketTimeoutException("Timed out reading from socket");
-                }
-                try {
-                    TimeUnit.NANOSECONDS.sleep(CHANNEL_EMPTY_WAIT_NANOS);
-                } catch (InterruptedException e) {
-                    close();
-                    Thread.currentThread().interrupt(); // set the interrupt status
-                    throw new ClosedByInterruptException(); // simulate an interrupted blocked read operation
-                }
-            }
-        } while (bytesRead == 0);
-
-        if (bytesRead == -1) {
-            return -1;
-        }
-        oneByteBuffer.flip();
-        return oneByteBuffer.get() & 0xFF;
-    }
-
-    @Override
-    public int read(final byte[] b) throws IOException {
-        return read(b, 0, b.length);
-    }
-
-    @Override
-    public int read(final byte[] b, final int off, final int len) throws IOException {
-        if (bufferedByte != null) {
-            final byte retVal = bufferedByte;
-            bufferedByte = null;
-            b[off] = retVal;
-            return 1;
-        }
-
-        final ByteBuffer buffer = ByteBuffer.wrap(b, off, len);
-
-        final long maxTime = System.currentTimeMillis() + timeoutMillis;
-        int bytesRead;
-        do {
-            bytesRead = channel.read(buffer);
-            if (bytesRead == 0) {
-                if (System.currentTimeMillis() > maxTime) {
-                    throw new SocketTimeoutException("Timed out reading from socket");
-                }
-                try {
-                    TimeUnit.NANOSECONDS.sleep(CHANNEL_EMPTY_WAIT_NANOS);
-                } catch (InterruptedException e) {
-                    close();
-                    Thread.currentThread().interrupt(); // set the interrupt status
-                    throw new ClosedByInterruptException(); // simulate an interrupted blocked read operation
-                }
-            }
-        } while (bytesRead == 0);
-
-        return bytesRead;
-    }
-
-    @Override
-    public int available() throws IOException {
-        if (bufferedByte != null) {
-            return 1;
-        }
-
-        isDataAvailable(); // attempt to read from socket
-        return (bufferedByte == null) ? 0 : 1;
-    }
-
-    public boolean isDataAvailable() throws IOException {
-        if (bufferedByte != null) {
-            return true;
-        }
-
-        oneByteBuffer.flip();
-        oneByteBuffer.clear();
-        final int bytesRead = channel.read(oneByteBuffer);
-        if (bytesRead == -1) {
-            throw new EOFException("Peer has closed the stream");
-        }
-        if (bytesRead > 0) {
-            oneByteBuffer.flip();
-            bufferedByte = oneByteBuffer.get();
-            return true;
-        }
-        return false;
-    }
-
-    /**
-     * Closes the underlying socket channel.
-     * @throws java.io.IOException
-     */
-    @Override
-    public void close() throws IOException {
-        channel.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutputStream.java
deleted file mode 100644
index 77049ad..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutputStream.java
+++ /dev/null
@@ -1,113 +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.remote.io.socket;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.net.SocketTimeoutException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ClosedByInterruptException;
-import java.nio.channels.SocketChannel;
-import java.util.concurrent.TimeUnit;
-
-public class SocketChannelOutputStream extends OutputStream {
-
-    private static final long CHANNEL_FULL_WAIT_NANOS = TimeUnit.NANOSECONDS.convert(10, TimeUnit.MILLISECONDS);
-    private final SocketChannel channel;
-    private volatile int timeout = 30000;
-
-    private final ByteBuffer oneByteBuffer = ByteBuffer.allocate(1);
-
-    public SocketChannelOutputStream(final SocketChannel socketChannel) throws IOException {
-        // this class expects a non-blocking channel
-        socketChannel.configureBlocking(false);
-        this.channel = socketChannel;
-    }
-
-    public void setTimeout(final int timeoutMillis) {
-        this.timeout = timeoutMillis;
-    }
-
-    @Override
-    public void write(final int b) throws IOException {
-        oneByteBuffer.flip();
-        oneByteBuffer.clear();
-        oneByteBuffer.put((byte) b);
-        oneByteBuffer.flip();
-
-        final int timeoutMillis = this.timeout;
-        long maxTime = System.currentTimeMillis() + timeoutMillis;
-        int bytesWritten;
-        while (oneByteBuffer.hasRemaining()) {
-            bytesWritten = channel.write(oneByteBuffer);
-            if (bytesWritten == 0) {
-                if (System.currentTimeMillis() > maxTime) {
-                    throw new SocketTimeoutException("Timed out writing to socket");
-                }
-                try {
-                    TimeUnit.NANOSECONDS.sleep(CHANNEL_FULL_WAIT_NANOS);
-                } catch (InterruptedException e) {
-                    close();
-                    Thread.currentThread().interrupt(); // set the interrupt status
-                    throw new ClosedByInterruptException(); // simulate an interrupted blocked write operation
-                }
-            } else {
-                return;
-            }
-        }
-    }
-
-    @Override
-    public void write(final byte[] b) throws IOException {
-        write(b, 0, b.length);
-    }
-
-    @Override
-    public void write(final byte[] b, final int off, final int len) throws IOException {
-        final ByteBuffer buffer = ByteBuffer.wrap(b, off, len);
-
-        final int timeoutMillis = this.timeout;
-        long maxTime = System.currentTimeMillis() + timeoutMillis;
-        int bytesWritten;
-        while (buffer.hasRemaining()) {
-            bytesWritten = channel.write(buffer);
-            if (bytesWritten == 0) {
-                if (System.currentTimeMillis() > maxTime) {
-                    throw new SocketTimeoutException("Timed out writing to socket");
-                }
-                try {
-                    TimeUnit.NANOSECONDS.sleep(CHANNEL_FULL_WAIT_NANOS);
-                } catch (InterruptedException e) {
-                    close();
-                    Thread.currentThread().interrupt(); // set the interrupt status
-                    throw new ClosedByInterruptException(); // simulate an interrupted blocked write operation
-                }
-            } else {
-                maxTime = System.currentTimeMillis() + timeoutMillis;
-            }
-        }
-    }
-
-    /**
-     * Closes the underlying SocketChannel
-     * @throws java.io.IOException
-     */
-    @Override
-    public void close() throws IOException {
-        channel.close();
-    }
-}


[28/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAction.java
deleted file mode 100644
index cef21d7..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAction.java
+++ /dev/null
@@ -1,124 +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.admin.service.action;
-
-import java.util.Date;
-import java.util.Set;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.authorization.Authority;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.exception.AuthorityAccessException;
-import org.apache.nifi.authorization.exception.IdentityAlreadyExistsException;
-import org.apache.nifi.authorization.exception.UnknownIdentityException;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUser;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Sets user authorities.
- */
-public class UpdateUserAction extends AbstractUserAction<NiFiUser> {
-
-    private static final Logger logger = LoggerFactory.getLogger(UpdateUserAction.class);
-
-    private final String id;
-    private final Set<Authority> authorities;
-
-    public UpdateUserAction(String id, Set<Authority> authorities) {
-        this.id = id;
-        this.authorities = authorities;
-    }
-
-    @Override
-    public NiFiUser execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException, AdministrationException {
-        UserDAO userDao = daoFactory.getUserDAO();
-
-        // get the user
-        NiFiUser user = userDao.findUserById(id);
-
-        // ensure the user exists
-        if (user == null) {
-            throw new AccountNotFoundException(String.format("Unable to find account with ID %s.", id));
-        }
-
-        // determine whether this users exists
-        boolean doesDnExist = false;
-        try {
-            doesDnExist = authorityProvider.doesDnExist(user.getDn());
-        } catch (AuthorityAccessException aae) {
-            throw new AdministrationException(String.format("Unable to access authority details: %s", aae.getMessage()), aae);
-        }
-
-        // if the user already doesn't exist, add them
-        if (!doesDnExist) {
-            try {
-                // add the account account and group if necessary
-                authorityProvider.addUser(user.getDn(), user.getUserGroup());
-            } catch (final IdentityAlreadyExistsException iaee) {
-                logger.warn(String.format("User '%s' already exists in the authority provider.  Continuing with user update.", user.getDn()));
-            } catch (AuthorityAccessException aae) {
-                throw new AdministrationException(String.format("Unable to access authorities for '%s': %s", user.getDn(), aae.getMessage()), aae);
-            }
-        }
-
-        try {
-            // update the authority provider as approprivate
-            authorityProvider.setAuthorities(user.getDn(), authorities);
-        } catch (UnknownIdentityException uie) {
-            throw new AccountNotFoundException(String.format("Unable to modify authorities for '%s': %s.", user.getDn(), uie.getMessage()), uie);
-        } catch (AuthorityAccessException aae) {
-            throw new AdministrationException(String.format("Unable to access authorities for '%s': %s.", user.getDn(), aae.getMessage()), aae);
-        }
-
-        try {
-            // get the user group
-            user.setUserGroup(authorityProvider.getGroupForUser(user.getDn()));
-        } catch (UnknownIdentityException uie) {
-            throw new AccountNotFoundException(String.format("Unable to determine the group for '%s': %s.", user.getDn(), uie.getMessage()), uie);
-        } catch (AuthorityAccessException aae) {
-            throw new AdministrationException(String.format("Unable to access the group for '%s': %s.", user.getDn(), aae.getMessage()), aae);
-        }
-
-        // since all the authorities were updated accordingly, set the authorities
-        user.getAuthorities().clear();
-        user.getAuthorities().addAll(authorities);
-
-        // update the users status in case they were previously pending or disabled
-        user.setStatus(AccountStatus.ACTIVE);
-
-        // update the users last verified time - this timestamp shouldn't be recorded
-        // until the both the user's authorities and group have been synced
-        Date now = new Date();
-        user.setLastVerified(now);
-
-        // persist the user's updates
-        UpdateUserCacheAction updateUser = new UpdateUserCacheAction(user);
-        updateUser.execute(daoFactory, authorityProvider);
-
-        // persist the user's authorities
-        UpdateUserAuthoritiesCacheAction updateUserAuthorities = new UpdateUserAuthoritiesCacheAction(user);
-        updateUserAuthorities.execute(daoFactory, authorityProvider);
-
-        // return the user
-        return user;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAuthoritiesCacheAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAuthoritiesCacheAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAuthoritiesCacheAction.java
deleted file mode 100644
index 89661b2..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAuthoritiesCacheAction.java
+++ /dev/null
@@ -1,73 +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.admin.service.action;
-
-import java.util.Set;
-import org.apache.nifi.admin.dao.AuthorityDAO;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.authorization.Authority;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.user.NiFiUser;
-import org.apache.commons.collections4.CollectionUtils;
-
-/**
- * Updates a NiFiUser's authorities. Prior to invoking this action, the user's
- * authorities should be set according to the business logic of the service in
- * question. This should not be invoked directly when attempting to set user
- * authorities as the authorityProvider is not called from this action.
- */
-public class UpdateUserAuthoritiesCacheAction extends AbstractUserAction<Void> {
-
-    private final NiFiUser user;
-
-    public UpdateUserAuthoritiesCacheAction(NiFiUser user) {
-        this.user = user;
-    }
-
-    @Override
-    public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
-        UserDAO userDao = daoFactory.getUserDAO();
-        AuthorityDAO authorityDao = daoFactory.getAuthorityDAO();
-
-        // get the user
-        NiFiUser currentUser = userDao.findUserById(user.getId());
-
-        // ensure the user exists
-        if (currentUser == null) {
-            throw new AccountNotFoundException(String.format("Unable to find account with ID %s.", user.getId()));
-        }
-
-        // determine what authorities need to be added/removed
-        Set<Authority> authorities = user.getAuthorities();
-        Set<Authority> authoritiesToAdd = determineAuthoritiesToAdd(currentUser, authorities);
-        Set<Authority> authoritiesToRemove = determineAuthoritiesToRemove(currentUser, authorities);
-
-        // update the user authorities locally
-        if (CollectionUtils.isNotEmpty(authoritiesToAdd)) {
-            authorityDao.createAuthorities(authoritiesToAdd, user.getId());
-        }
-        if (CollectionUtils.isNotEmpty(authoritiesToRemove)) {
-            authorityDao.deleteAuthorities(authoritiesToRemove, user.getId());
-        }
-
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserCacheAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserCacheAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserCacheAction.java
deleted file mode 100644
index 288e297..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserCacheAction.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.admin.service.action;
-
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.user.NiFiUser;
-
-/**
- * Updates a NiFiUser. This will not update the user authorities, they must be
- * updated with the UpdateUserAuthoritiesAction.
- */
-public class UpdateUserCacheAction extends AbstractUserAction<Void> {
-
-    private final NiFiUser user;
-
-    public UpdateUserCacheAction(NiFiUser user) {
-        this.user = user;
-    }
-
-    @Override
-    public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
-        UserDAO userDao = daoFactory.getUserDAO();
-
-        // update the user
-        userDao.updateUser(user);
-
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserGroupAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserGroupAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserGroupAction.java
deleted file mode 100644
index 56b214c..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserGroupAction.java
+++ /dev/null
@@ -1,171 +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.admin.service.action;
-
-import java.util.Date;
-import java.util.HashSet;
-import java.util.Set;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.authorization.Authority;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.exception.AuthorityAccessException;
-import org.apache.nifi.authorization.exception.UnknownIdentityException;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUser;
-import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Updates all NiFiUser authorities in a specified group.
- */
-public class UpdateUserGroupAction extends AbstractUserAction<Void> {
-
-    private static final Logger logger = LoggerFactory.getLogger(UpdateUserGroupAction.class);
-
-    private final String group;
-    private final Set<String> userIds;
-    private final Set<Authority> authorities;
-
-    public UpdateUserGroupAction(String group, Set<String> userIds, Set<Authority> authorities) {
-        this.group = group;
-        this.userIds = userIds;
-        this.authorities = authorities;
-    }
-
-    @Override
-    public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException {
-        if (userIds == null && authorities == null) {
-            throw new IllegalArgumentException("Must specify user Ids or authorities.");
-        }
-
-        UserDAO userDao = daoFactory.getUserDAO();
-
-        // record the new users being added to this group
-        final Set<NiFiUser> newUsers = new HashSet<>();
-        final Set<String> newUserDns = new HashSet<>();
-
-        // if the user ids have been specified we need to create/update a group using the specified group name
-        if (userIds != null) {
-            if (userIds.isEmpty()) {
-                throw new IllegalArgumentException("When creating a group, at least one user id must be specified.");
-            }
-
-            // going to create a group using the specified user ids
-            for (final String userId : userIds) {
-                // get the user in question
-                final NiFiUser user = userDao.findUserById(userId);
-
-                // ensure the user exists
-                if (user == null) {
-                    throw new AccountNotFoundException(String.format("Unable to find account with ID %s.", userId));
-                }
-
-                try {
-                    // if the user is unknown to the authority provider we cannot continue
-                    if (!authorityProvider.doesDnExist(user.getDn()) || AccountStatus.DISABLED.equals(user.getStatus())) {
-                        throw new IllegalStateException(String.format("Unable to group these users because access for '%s' is not %s.", user.getDn(), AccountStatus.ACTIVE.toString()));
-                    }
-
-                    // record the user being added to this group
-                    newUsers.add(user);
-                    newUserDns.add(user.getDn());
-                } catch (final AuthorityAccessException aae) {
-                    throw new AdministrationException(String.format("Unable to access authority details: %s", aae.getMessage()), aae);
-                }
-            }
-
-            try {
-                // update the authority provider
-                authorityProvider.setUsersGroup(newUserDns, group);
-            } catch (UnknownIdentityException uie) {
-                throw new AccountNotFoundException(String.format("Unable to set user group '%s': %s", StringUtils.join(newUserDns, ", "), uie.getMessage()), uie);
-            } catch (AuthorityAccessException aae) {
-                throw new AdministrationException(String.format("Unable to set user group '%s': %s", StringUtils.join(newUserDns, ", "), aae.getMessage()), aae);
-            }
-        }
-
-        // get all the users that need to be updated
-        final Set<NiFiUser> users = new HashSet<>(userDao.findUsersForGroup(group));
-        users.addAll(newUsers);
-
-        // ensure the user exists
-        if (users.isEmpty()) {
-            throw new AccountNotFoundException(String.format("Unable to find user accounts with group id %s.", group));
-        }
-
-        // update each user in this group
-        for (final NiFiUser user : users) {
-            // if there are new authorities set them, otherwise refresh them according to the provider
-            if (authorities != null) {
-                try {
-                    // update the authority provider as approprivate
-                    authorityProvider.setAuthorities(user.getDn(), authorities);
-
-                    // since all the authorities were updated accordingly, set the authorities
-                    user.getAuthorities().clear();
-                    user.getAuthorities().addAll(authorities);
-                } catch (UnknownIdentityException uie) {
-                    throw new AccountNotFoundException(String.format("Unable to modify authorities for '%s': %s.", user.getDn(), uie.getMessage()), uie);
-                } catch (AuthorityAccessException aae) {
-                    throw new AdministrationException(String.format("Unable to access authorities for '%s': %s.", user.getDn(), aae.getMessage()), aae);
-                }
-            } else {
-                try {
-                    // refresh the authorities according to the provider
-                    user.getAuthorities().clear();
-                    user.getAuthorities().addAll(authorityProvider.getAuthorities(user.getDn()));
-                } catch (UnknownIdentityException uie) {
-                    throw new AccountNotFoundException(String.format("Unable to determine the authorities for '%s': %s.", user.getDn(), uie.getMessage()), uie);
-                } catch (AuthorityAccessException aae) {
-                    throw new AdministrationException(String.format("Unable to access authorities for '%s': %s.", user.getDn(), aae.getMessage()), aae);
-                }
-            }
-
-            try {
-                // get the user group
-                user.setUserGroup(authorityProvider.getGroupForUser(user.getDn()));
-            } catch (UnknownIdentityException uie) {
-                throw new AccountNotFoundException(String.format("Unable to determine the group for '%s': %s.", user.getDn(), uie.getMessage()), uie);
-            } catch (AuthorityAccessException aae) {
-                throw new AdministrationException(String.format("Unable to access the group for '%s': %s.", user.getDn(), aae.getMessage()), aae);
-            }
-
-            // update the users status in case they were previously pending or disabled
-            user.setStatus(AccountStatus.ACTIVE);
-
-            // update the users last verified time - this timestamp shouldn't be recorded
-            // until the both the user's authorities and group have been synced
-            Date now = new Date();
-            user.setLastVerified(now);
-
-            // persist the user's updates
-            UpdateUserCacheAction updateUser = new UpdateUserCacheAction(user);
-            updateUser.execute(daoFactory, authorityProvider);
-
-            // persist the user's authorities
-            UpdateUserAuthoritiesCacheAction updateUserAuthorities = new UpdateUserAuthoritiesCacheAction(user);
-            updateUserAuthorities.execute(daoFactory, authorityProvider);
-        }
-
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java
deleted file mode 100644
index 127f1df..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java
+++ /dev/null
@@ -1,230 +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.admin.service.impl;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.apache.nifi.action.Action;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.admin.service.AuditService;
-import org.apache.nifi.admin.service.action.AddActionsAction;
-import org.apache.nifi.admin.service.action.GetActionAction;
-import org.apache.nifi.admin.service.action.GetActionsAction;
-import org.apache.nifi.admin.service.action.GetPreviousValues;
-import org.apache.nifi.admin.service.action.PurgeActionsAction;
-import org.apache.nifi.admin.service.transaction.Transaction;
-import org.apache.nifi.admin.service.transaction.TransactionBuilder;
-import org.apache.nifi.admin.service.transaction.TransactionException;
-import org.apache.nifi.history.History;
-import org.apache.nifi.history.HistoryQuery;
-import org.apache.nifi.history.PreviousValue;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- *
- */
-public class StandardAuditService implements AuditService {
-
-    private static final Logger logger = LoggerFactory.getLogger(StandardAuditService.class);
-
-    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
-    private final ReentrantReadWriteLock.ReadLock readLock = lock.readLock();
-    private final ReentrantReadWriteLock.WriteLock writeLock = lock.writeLock();
-
-    private TransactionBuilder transactionBuilder;
-
-    @Override
-    public void addActions(Collection<Action> actions) {
-        Transaction transaction = null;
-
-        writeLock.lock();
-        try {
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            // seed the accounts
-            AddActionsAction addActions = new AddActionsAction(actions);
-            transaction.execute(addActions);
-
-            // commit the transaction
-            transaction.commit();
-        } catch (TransactionException | DataAccessException te) {
-            rollback(transaction);
-            throw new AdministrationException(te);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public Map<String, List<PreviousValue>> getPreviousValues(String processorId) {
-        Transaction transaction = null;
-        Map<String, List<PreviousValue>> previousValues = null;
-
-        readLock.lock();
-        try {
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            // seed the accounts
-            GetPreviousValues getActions = new GetPreviousValues(processorId);
-            previousValues = transaction.execute(getActions);
-
-            // commit the transaction
-            transaction.commit();
-        } catch (TransactionException | DataAccessException te) {
-            rollback(transaction);
-            throw new AdministrationException(te);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            readLock.unlock();
-        }
-
-        return previousValues;
-    }
-
-    @Override
-    public History getActions(HistoryQuery query) {
-        Transaction transaction = null;
-        History history = null;
-
-        readLock.lock();
-        try {
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            // seed the accounts
-            GetActionsAction getActions = new GetActionsAction(query);
-            history = transaction.execute(getActions);
-
-            // commit the transaction
-            transaction.commit();
-        } catch (TransactionException | DataAccessException te) {
-            rollback(transaction);
-            throw new AdministrationException(te);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            readLock.unlock();
-        }
-
-        return history;
-    }
-
-    @Override
-    public Action getAction(Integer actionId) {
-        Transaction transaction = null;
-        Action action = null;
-
-        readLock.lock();
-        try {
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            // seed the accounts
-            GetActionAction getAction = new GetActionAction(actionId);
-            action = transaction.execute(getAction);
-
-            // commit the transaction
-            transaction.commit();
-        } catch (TransactionException | DataAccessException te) {
-            rollback(transaction);
-            throw new AdministrationException(te);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            readLock.unlock();
-        }
-
-        return action;
-    }
-
-    @Override
-    public void purgeActions(Date end, Action purgeAction) {
-        Transaction transaction = null;
-
-        writeLock.lock();
-        try {
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            // purge the action database
-            PurgeActionsAction purgeActions = new PurgeActionsAction(end, purgeAction);
-            transaction.execute(purgeActions);
-
-            // commit the transaction
-            transaction.commit();
-        } catch (TransactionException | DataAccessException te) {
-            rollback(transaction);
-            throw new AdministrationException(te);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * Rolls back the specified transaction.
-     *
-     * @param transaction
-     */
-    private void rollback(Transaction transaction) {
-        if (transaction != null) {
-            transaction.rollback();
-        }
-    }
-
-    /**
-     * Closes the specified transaction.
-     *
-     * @param transaction
-     */
-    private void closeQuietly(final Transaction transaction) {
-        if (transaction != null) {
-            try {
-                transaction.close();
-            } catch (final IOException ioe) {
-            }
-        }
-    }
-
-    /* setters */
-    public void setTransactionBuilder(TransactionBuilder transactionBuilder) {
-        this.transactionBuilder = transactionBuilder;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardUserService.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardUserService.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardUserService.java
deleted file mode 100644
index 5c9af4b..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardUserService.java
+++ /dev/null
@@ -1,663 +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.admin.service.impl;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.service.AccountDisabledException;
-import org.apache.nifi.admin.service.AccountPendingException;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.admin.service.UserService;
-import org.apache.nifi.admin.service.action.AuthorizeDownloadAction;
-import org.apache.nifi.admin.service.action.AuthorizeUserAction;
-import org.apache.nifi.admin.service.action.DeleteUserAction;
-import org.apache.nifi.admin.service.action.DisableUserAction;
-import org.apache.nifi.admin.service.action.DisableUserGroupAction;
-import org.apache.nifi.admin.service.action.FindUserByDnAction;
-import org.apache.nifi.admin.service.action.FindUserByIdAction;
-import org.apache.nifi.admin.service.action.GetUserGroupAction;
-import org.apache.nifi.admin.service.action.GetUsersAction;
-import org.apache.nifi.admin.service.action.HasPendingUserAccounts;
-import org.apache.nifi.admin.service.action.InvalidateUserAccountAction;
-import org.apache.nifi.admin.service.action.InvalidateUserGroupAccountsAction;
-import org.apache.nifi.admin.service.action.RequestUserAccountAction;
-import org.apache.nifi.admin.service.action.SeedUserAccountsAction;
-import org.apache.nifi.admin.service.action.UpdateUserAction;
-import org.apache.nifi.admin.service.action.UpdateUserGroupAction;
-import org.apache.nifi.admin.service.action.UngroupUserAction;
-import org.apache.nifi.admin.service.action.UngroupUserGroupAction;
-import org.apache.nifi.admin.service.transaction.Transaction;
-import org.apache.nifi.admin.service.transaction.TransactionBuilder;
-import org.apache.nifi.admin.service.transaction.TransactionException;
-import org.apache.nifi.authorization.Authority;
-import org.apache.nifi.authorization.DownloadAuthorization;
-import org.apache.nifi.user.NiFiUser;
-import org.apache.nifi.user.NiFiUserGroup;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.NiFiProperties;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- *
- */
-public class StandardUserService implements UserService {
-
-    private static final Logger logger = LoggerFactory.getLogger(StandardUserService.class);
-
-    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
-    private final ReentrantReadWriteLock.ReadLock readLock = lock.readLock();
-    private final ReentrantReadWriteLock.WriteLock writeLock = lock.writeLock();
-
-    private TransactionBuilder transactionBuilder;
-    private NiFiProperties properties;
-
-    /**
-     * Seed any users from the authority provider that are not already present.
-     */
-    public void seedUserAccounts() {
-        // do not seed node's user cache. when/if the node disconnects its 
-        // cache will be populated lazily (as needed)
-        if (properties.isNode()) {
-            return;
-        }
-
-        Transaction transaction = null;
-        writeLock.lock();
-        try {
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            // seed the accounts
-            SeedUserAccountsAction seedUserAccounts = new SeedUserAccountsAction();
-            transaction.execute(seedUserAccounts);
-
-            // commit the transaction
-            transaction.commit();
-        } catch (AdministrationException ae) {
-            rollback(transaction);
-            throw ae;
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public NiFiUser createPendingUserAccount(String dn, String justification) {
-        Transaction transaction = null;
-
-        writeLock.lock();
-        try {
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            // create the account request
-            RequestUserAccountAction requestUserAccount = new RequestUserAccountAction(dn, justification);
-            NiFiUser user = transaction.execute(requestUserAccount);
-
-            // commit the transaction
-            transaction.commit();
-
-            // return the nifi user
-            return user;
-        } catch (TransactionException | DataAccessException te) {
-            rollback(transaction);
-            throw new AdministrationException(te);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public NiFiUserGroup updateGroup(final String group, final Set<String> userIds, final Set<Authority> authorities) {
-        Transaction transaction = null;
-
-        writeLock.lock();
-        try {
-            // if user ids have been specified, invalidate the user accounts before performing 
-            // the desired updates. if case of an error, this will ensure that these users are
-            // authorized the next time the access the application
-            if (userIds != null) {
-                for (final String userId : userIds) {
-                    invalidateUserAccount(userId);
-                }
-            }
-
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            // set the authorities for each user in this group if specified
-            final UpdateUserGroupAction updateUserGroup = new UpdateUserGroupAction(group, userIds, authorities);
-            transaction.execute(updateUserGroup);
-
-            // get all the users that are now in this group
-            final GetUserGroupAction getUserGroup = new GetUserGroupAction(group);
-            final NiFiUserGroup userGroup = transaction.execute(getUserGroup);
-
-            // commit the transaction
-            transaction.commit();
-
-            return userGroup;
-        } catch (TransactionException | DataAccessException te) {
-            rollback(transaction);
-            throw new AdministrationException(te);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public void ungroupUser(String id) {
-        Transaction transaction = null;
-
-        writeLock.lock();
-        try {
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            // ungroup the specified user
-            final UngroupUserAction ungroupUser = new UngroupUserAction(id);
-            transaction.execute(ungroupUser);
-
-            // commit the transaction
-            transaction.commit();
-        } catch (TransactionException | DataAccessException te) {
-            rollback(transaction);
-            throw new AdministrationException(te);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public void ungroup(String group) {
-        Transaction transaction = null;
-
-        writeLock.lock();
-        try {
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            // ungroup the specified user
-            final UngroupUserGroupAction ungroupUserGroup = new UngroupUserGroupAction(group);
-            transaction.execute(ungroupUserGroup);
-
-            // commit the transaction
-            transaction.commit();
-        } catch (TransactionException | DataAccessException te) {
-            rollback(transaction);
-            throw new AdministrationException(te);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public NiFiUser checkAuthorization(String dn) {
-        Transaction transaction = null;
-
-        writeLock.lock();
-        try {
-            // create the connection
-            transaction = transactionBuilder.start();
-
-            // determine how long the cache is valid for
-            final int cacheSeconds;
-            try {
-                cacheSeconds = (int) FormatUtils.getTimeDuration(properties.getUserCredentialCacheDuration(), TimeUnit.SECONDS);
-            } catch (IllegalArgumentException iae) {
-                throw new AdministrationException("User credential cache duration is not configured correctly.");
-            }
-
-            // attempt to authorize the user
-            AuthorizeUserAction authorizeUser = new AuthorizeUserAction(dn, cacheSeconds);
-            NiFiUser user = transaction.execute(authorizeUser);
-
-            // commit the transaction
-            transaction.commit();
-
-            // return the nifi user
-            return user;
-        } catch (DataAccessException | TransactionException dae) {
-            rollback(transaction);
-            throw new AdministrationException(dae);
-        } catch (AccountDisabledException | AccountPendingException ade) {
-            rollback(transaction);
-            throw ade;
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public void deleteUser(String id) {
-        Transaction transaction = null;
-
-        writeLock.lock();
-        try {
-            // create the connection
-            transaction = transactionBuilder.start();
-
-            // delete the user
-            DeleteUserAction deleteUser = new DeleteUserAction(id);
-            transaction.execute(deleteUser);
-
-            // commit the transaction
-            transaction.commit();
-        } catch (DataAccessException | TransactionException dae) {
-            rollback(transaction);
-            throw new AdministrationException(dae);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public NiFiUser disable(String id) {
-        Transaction transaction = null;
-
-        writeLock.lock();
-        try {
-            // create the connection
-            transaction = transactionBuilder.start();
-
-            // disable the user
-            DisableUserAction disableUser = new DisableUserAction(id);
-            NiFiUser user = transaction.execute(disableUser);
-
-            // commit the transaction
-            transaction.commit();
-
-            // return the user
-            return user;
-        } catch (DataAccessException | TransactionException dae) {
-            rollback(transaction);
-            throw new AdministrationException(dae);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public NiFiUserGroup disableGroup(String group) {
-        Transaction transaction = null;
-
-        writeLock.lock();
-        try {
-            // create the connection
-            transaction = transactionBuilder.start();
-
-            // disable the user
-            DisableUserGroupAction disableUser = new DisableUserGroupAction(group);
-            NiFiUserGroup userGroup = transaction.execute(disableUser);
-
-            // commit the transaction
-            transaction.commit();
-
-            // return the user
-            return userGroup;
-        } catch (DataAccessException | TransactionException dae) {
-            rollback(transaction);
-            throw new AdministrationException(dae);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public NiFiUser update(String id, Set<Authority> authorities) {
-        Transaction transaction = null;
-
-        // may be empty but not null
-        if (authorities == null) {
-            throw new IllegalArgumentException("The specified authorities cannot be null.");
-        }
-
-        writeLock.lock();
-        try {
-            // invalidate the user account in preparation for potential subsequent errors
-            invalidateUserAccount(id);
-
-            // at this point the current user account has been invalidated so we will
-            // attempt to update the account. if any part fails we are assured the
-            // user will be need to be given approval before they access the system at
-            // a later time
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            // update the user authorities
-            UpdateUserAction setUserAuthorities = new UpdateUserAction(id, authorities);
-            NiFiUser user = transaction.execute(setUserAuthorities);
-
-            // commit the transaction
-            transaction.commit();
-
-            // return the user
-            return user;
-        } catch (TransactionException | DataAccessException e) {
-            rollback(transaction);
-            throw new AdministrationException(e);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * Invalidates the user with the specified id. This is done to ensure a user
-     * account will need to be re-validated in case an error occurs while
-     * modifying a user account. This method should only be invoked from within
-     * a write lock.
-     *
-     * @param id
-     */
-    @Override
-    public void invalidateUserAccount(String id) {
-        Transaction transaction = null;
-
-        writeLock.lock();
-        try {
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            // invalidate the user account
-            InvalidateUserAccountAction invalidateUserAccount = new InvalidateUserAccountAction(id);
-            transaction.execute(invalidateUserAccount);
-
-            // commit the transaction
-            transaction.commit();
-        } catch (TransactionException | DataAccessException te) {
-            rollback(transaction);
-            throw new AdministrationException(te);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * Invalidates the user with the specified id. This is done to ensure a user
-     * account will need to be re-validated in case an error occurs while
-     * modifying a user account. This method should only be invoked from within
-     * a write lock.
-     *
-     * @param group
-     */
-    @Override
-    public void invalidateUserGroupAccount(String group) {
-        Transaction transaction = null;
-
-        writeLock.lock();
-        try {
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            // invalidate the user account
-            InvalidateUserGroupAccountsAction invalidateUserGroupAccounts = new InvalidateUserGroupAccountsAction(group);
-            transaction.execute(invalidateUserGroupAccounts);
-
-            // commit the transaction
-            transaction.commit();
-        } catch (TransactionException | DataAccessException te) {
-            rollback(transaction);
-            throw new AdministrationException(te);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            writeLock.unlock();
-        }
-    }
-
-    // -----------------
-    // read only methods
-    // -----------------
-    @Override
-    public Boolean hasPendingUserAccount() {
-        Transaction transaction = null;
-
-        readLock.lock();
-        try {
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            final HasPendingUserAccounts hasPendingAccounts = new HasPendingUserAccounts();
-            final Boolean hasPendingUserAccounts = transaction.execute(hasPendingAccounts);
-
-            // commit the transaction
-            transaction.commit();
-
-            return hasPendingUserAccounts;
-        } catch (TransactionException | DataAccessException te) {
-            rollback(transaction);
-            throw new AdministrationException(te);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public DownloadAuthorization authorizeDownload(final List<String> dnChain, final Map<String, String> attributes) {
-        Transaction transaction = null;
-
-        readLock.lock();
-        try {
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            // authorize the download
-            AuthorizeDownloadAction authorizeDownload = new AuthorizeDownloadAction(dnChain, attributes);
-            DownloadAuthorization downloadAuthorization = transaction.execute(authorizeDownload);
-
-            // commit the transaction
-            transaction.commit();
-
-            // return the authorization
-            return downloadAuthorization;
-        } catch (TransactionException | DataAccessException te) {
-            rollback(transaction);
-            throw new AdministrationException(te);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            readLock.unlock();
-        }
-    }
-    
-    @Override
-    public Collection<NiFiUser> getUsers() {
-        Transaction transaction = null;
-
-        readLock.lock();
-        try {
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            // get all users
-            GetUsersAction getUsers = new GetUsersAction();
-            Collection<NiFiUser> users = transaction.execute(getUsers);
-
-            // commit the transaction
-            transaction.commit();
-
-            // return the users
-            return users;
-        } catch (TransactionException | DataAccessException te) {
-            rollback(transaction);
-            throw new AdministrationException(te);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public NiFiUser getUserById(String id) {
-        Transaction transaction = null;
-
-        readLock.lock();
-        try {
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            // return the desired user
-            FindUserByIdAction findUserById = new FindUserByIdAction(id);
-            NiFiUser user = transaction.execute(findUserById);
-
-            // commit the transaction
-            transaction.commit();
-
-            // return the user
-            return user;
-        } catch (TransactionException | DataAccessException te) {
-            rollback(transaction);
-            throw new AdministrationException(te);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public NiFiUser getUserByDn(String dn) {
-        Transaction transaction = null;
-
-        readLock.lock();
-        try {
-            // start the transaction
-            transaction = transactionBuilder.start();
-
-            // return the desired user
-            FindUserByDnAction findUserByDn = new FindUserByDnAction(dn);
-            NiFiUser user = transaction.execute(findUserByDn);
-
-            // commit the transaction
-            transaction.commit();
-
-            // return the user
-            return user;
-        } catch (TransactionException | DataAccessException te) {
-            rollback(transaction);
-            throw new AdministrationException(te);
-        } catch (Throwable t) {
-            rollback(transaction);
-            throw t;
-        } finally {
-            closeQuietly(transaction);
-            readLock.unlock();
-        }
-    }
-
-    /**
-     * Rolls back the specified transaction.
-     *
-     * @param transaction
-     */
-    private void rollback(final Transaction transaction) {
-        if (transaction != null) {
-            transaction.rollback();
-        }
-    }
-
-    /**
-     * Closes the specified transaction.
-     *
-     * @param transaction
-     */
-    private void closeQuietly(final Transaction transaction) {
-        if (transaction != null) {
-            try {
-                transaction.close();
-            } catch (final IOException ioe) {
-            }
-        }
-    }
-
-    /*
-     * setters
-     */
-    public void setTransactionBuilder(TransactionBuilder transactionBuilder) {
-        this.transactionBuilder = transactionBuilder;
-    }
-
-    public void setProperties(NiFiProperties properties) {
-        this.properties = properties;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/Transaction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/Transaction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/Transaction.java
deleted file mode 100644
index edd214b..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/Transaction.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.admin.service.transaction;
-
-import java.io.Closeable;
-import org.apache.nifi.admin.service.action.AdministrationAction;
-
-/**
- * Defines a transaction.
- */
-public interface Transaction extends Closeable {
-
-    /**
-     * Executes the specified action within the current transaction.
-     *
-     * @param <T>
-     * @param action
-     * @return
-     * @throws IllegalStateException - if there is no current transaction
-     */
-    <T> T execute(AdministrationAction<T> action);
-
-    /**
-     * Commits the current transaction.
-     *
-     * @throws TransactionException - if the transaction is unable to be
-     * committed
-     */
-    void commit() throws TransactionException;
-
-    /**
-     * Rolls back the current transaction.
-     */
-    void rollback();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionBuilder.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionBuilder.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionBuilder.java
deleted file mode 100644
index 2d2ef82..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionBuilder.java
+++ /dev/null
@@ -1,25 +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.admin.service.transaction;
-
-/**
- *
- */
-public interface TransactionBuilder {
-
-    Transaction start() throws TransactionException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionException.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionException.java
deleted file mode 100644
index 924e01f..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionException.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.admin.service.transaction;
-
-/**
- * Exception to indicate that the user account is disabled.
- */
-public class TransactionException extends RuntimeException {
-
-    public TransactionException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
-        super(message, cause, enableSuppression, writableStackTrace);
-    }
-
-    public TransactionException(Throwable cause) {
-        super(cause);
-    }
-
-    public TransactionException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public TransactionException(String message) {
-        super(message);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransaction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransaction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransaction.java
deleted file mode 100644
index a3cfb5e..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransaction.java
+++ /dev/null
@@ -1,93 +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.admin.service.transaction.impl;
-
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.SQLException;
-import org.apache.nifi.admin.RepositoryUtils;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.impl.DAOFactoryImpl;
-import org.apache.nifi.admin.service.action.AdministrationAction;
-import org.apache.nifi.admin.service.transaction.TransactionException;
-import org.apache.nifi.admin.service.transaction.Transaction;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Transaction implementation that uses the specified SQL Connection and
- * AuthorityProvider.
- */
-public class StandardTransaction implements Transaction {
-
-    private static final Logger logger = LoggerFactory.getLogger(StandardTransaction.class);
-
-    private final AuthorityProvider authorityProvider;
-    private Connection connection;
-
-    public StandardTransaction(AuthorityProvider authorityProvider, Connection connection) {
-        this.authorityProvider = authorityProvider;
-        this.connection = connection;
-    }
-
-    @Override
-    public <T> T execute(AdministrationAction<T> action) {
-        // ensure the transaction has been started
-        if (connection == null) {
-            throw new IllegalStateException("This transaction is not active.");
-        }
-
-        // create a dao factory
-        DAOFactory daoFactory = new DAOFactoryImpl(connection);
-
-        // execute the specified action
-        return action.execute(daoFactory, authorityProvider);
-    }
-
-    @Override
-    public void commit() throws TransactionException {
-        // ensure there is an active transaction
-        if (connection == null) {
-            throw new IllegalStateException("No active transaction.");
-        }
-
-        try {
-            // commit the transaction
-            connection.commit();
-        } catch (SQLException sqle) {
-            throw new TransactionException(sqle.getMessage());
-        }
-    }
-
-    @Override
-    public void rollback() {
-        // ensure there is an active transaction
-        if (connection != null) {
-            // rollback the transaction
-            RepositoryUtils.rollback(connection, logger);
-        }
-    }
-
-    @Override
-    public void close() throws IOException {
-        if (connection != null) {
-            RepositoryUtils.closeQuietly(connection);
-            connection = null;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransactionBuilder.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransactionBuilder.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransactionBuilder.java
deleted file mode 100644
index b6e5a30..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransactionBuilder.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.admin.service.transaction.impl;
-
-import java.sql.Connection;
-import java.sql.SQLException;
-import javax.sql.DataSource;
-import org.apache.nifi.admin.service.transaction.Transaction;
-import org.apache.nifi.admin.service.transaction.TransactionBuilder;
-import org.apache.nifi.admin.service.transaction.TransactionException;
-import org.apache.nifi.authorization.AuthorityProvider;
-
-/**
- *
- */
-public class StandardTransactionBuilder implements TransactionBuilder {
-
-    private DataSource dataSource;
-    private AuthorityProvider authorityProvider;
-
-    @Override
-    public Transaction start() throws TransactionException {
-        try {
-            // get a new connection
-            Connection connection = dataSource.getConnection();
-            connection.setAutoCommit(false);
-
-            // create a new transaction
-            return new StandardTransaction(authorityProvider, connection);
-        } catch (SQLException sqle) {
-            throw new TransactionException(sqle.getMessage());
-        }
-    }
-
-    /* setters */
-    public void setDataSource(DataSource dataSource) {
-        this.dataSource = dataSource;
-    }
-
-    public void setAuthorityProvider(AuthorityProvider authorityProvider) {
-        this.authorityProvider = authorityProvider;
-    }
-}


[39/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java b/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
deleted file mode 100644
index 5810488..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
+++ /dev/null
@@ -1,602 +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.remote.io.socket.ssl;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.net.SocketAddress;
-import java.net.SocketTimeoutException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ClosedByInterruptException;
-import java.nio.channels.SocketChannel;
-import java.util.concurrent.TimeUnit;
-
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLEngine;
-import javax.net.ssl.SSLEngineResult;
-import javax.net.ssl.SSLEngineResult.Status;
-import javax.net.ssl.SSLHandshakeException;
-import javax.net.ssl.SSLPeerUnverifiedException;
-import javax.security.cert.CertificateExpiredException;
-import javax.security.cert.CertificateNotYetValidException;
-import javax.security.cert.X509Certificate;
-
-import org.apache.nifi.remote.exception.TransmissionDisabledException;
-import org.apache.nifi.remote.io.socket.BufferStateManager;
-import org.apache.nifi.remote.io.socket.BufferStateManager.Direction;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class SSLSocketChannel implements Closeable {
-
-    public static final int MAX_WRITE_SIZE = 65536;
-
-    private static final Logger logger = LoggerFactory.getLogger(SSLSocketChannel.class);
-    private static final long BUFFER_FULL_EMPTY_WAIT_NANOS = TimeUnit.NANOSECONDS.convert(10, TimeUnit.MILLISECONDS);
-
-    private final String hostname;
-    private final int port;
-    private final SSLEngine engine;
-    private final SocketAddress socketAddress;
-
-    private BufferStateManager streamInManager;
-    private BufferStateManager streamOutManager;
-    private BufferStateManager appDataManager;
-
-    private SocketChannel channel;
-
-    private final byte[] oneByteBuffer = new byte[1];
-
-    private int timeoutMillis = 30000;
-    private volatile boolean connected = false;
-    private boolean handshaking = false;
-    private boolean closed = false;
-    private volatile boolean interrupted = false;
-
-    public SSLSocketChannel(final SSLContext sslContext, final String hostname, final int port, final boolean client) throws IOException {
-        this.socketAddress = new InetSocketAddress(hostname, port);
-        this.channel = SocketChannel.open();
-        this.hostname = hostname;
-        this.port = port;
-        this.engine = sslContext.createSSLEngine();
-        this.engine.setUseClientMode(client);
-        engine.setNeedClientAuth(true);
-
-        streamInManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getPacketBufferSize()));
-        streamOutManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getPacketBufferSize()));
-        appDataManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getApplicationBufferSize()));
-    }
-
-    public SSLSocketChannel(final SSLContext sslContext, final SocketChannel socketChannel, final boolean client) throws IOException {
-        if (!socketChannel.isConnected()) {
-            throw new IllegalArgumentException("Cannot pass an un-connected SocketChannel");
-        }
-
-        this.channel = socketChannel;
-
-        this.socketAddress = socketChannel.getRemoteAddress();
-        final Socket socket = socketChannel.socket();
-        this.hostname = socket.getInetAddress().getHostName();
-        this.port = socket.getPort();
-
-        this.engine = sslContext.createSSLEngine();
-        this.engine.setUseClientMode(client);
-        engine.setNeedClientAuth(true);
-
-        streamInManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getPacketBufferSize()));
-        streamOutManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getPacketBufferSize()));
-        appDataManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getApplicationBufferSize()));
-    }
-
-    public void setTimeout(final int millis) {
-        this.timeoutMillis = millis;
-    }
-
-    public int getTimeout() {
-        return timeoutMillis;
-    }
-
-    public void connect() throws SSLHandshakeException, IOException {
-        try {
-            channel.configureBlocking(false);
-            if (!channel.isConnected()) {
-                final long startTime = System.currentTimeMillis();
-
-                if (!channel.connect(socketAddress)) {
-                    while (!channel.finishConnect()) {
-                        if (interrupted) {
-                            throw new TransmissionDisabledException();
-                        }
-                        if (System.currentTimeMillis() > startTime + timeoutMillis) {
-                            throw new SocketTimeoutException("Timed out connecting to " + hostname + ":" + port);
-                        }
-
-                        try {
-                            Thread.sleep(50L);
-                        } catch (final InterruptedException e) {
-                        }
-                    }
-                }
-            }
-            engine.beginHandshake();
-
-            performHandshake();
-            logger.debug("{} Successfully completed SSL handshake", this);
-
-            streamInManager.clear();
-            streamOutManager.clear();
-            appDataManager.clear();
-
-            connected = true;
-        } catch (final Exception e) {
-            logger.error("{} Failed to connect due to {}", this, e);
-            if (logger.isDebugEnabled()) {
-                logger.error("", e);
-            }
-            closeQuietly(channel);
-            engine.closeInbound();
-            engine.closeOutbound();
-            throw e;
-        }
-    }
-
-    public String getDn() throws CertificateExpiredException, CertificateNotYetValidException, SSLPeerUnverifiedException {
-        final X509Certificate[] certs = engine.getSession().getPeerCertificateChain();
-        if (certs == null || certs.length == 0) {
-            throw new SSLPeerUnverifiedException("No certificates found");
-        }
-
-        final X509Certificate cert = certs[0];
-        cert.checkValidity();
-        return cert.getSubjectDN().getName().trim();
-    }
-
-    private void performHandshake() throws IOException {
-        // Generate handshake message
-        final byte[] emptyMessage = new byte[0];
-        handshaking = true;
-        logger.debug("{} Performing Handshake", this);
-
-        try {
-            while (true) {
-                switch (engine.getHandshakeStatus()) {
-                    case FINISHED:
-                        return;
-                    case NEED_WRAP: {
-                        final ByteBuffer appDataOut = ByteBuffer.wrap(emptyMessage);
-
-                        final ByteBuffer outboundBuffer = streamOutManager.prepareForWrite(engine.getSession().getApplicationBufferSize());
-
-                        final SSLEngineResult wrapHelloResult = engine.wrap(appDataOut, outboundBuffer);
-                        if (wrapHelloResult.getStatus() == Status.BUFFER_OVERFLOW) {
-                            streamOutManager.prepareForWrite(engine.getSession().getApplicationBufferSize());
-                            continue;
-                        }
-
-                        if (wrapHelloResult.getStatus() != Status.OK) {
-                            throw new SSLHandshakeException("Could not generate SSL Handshake information: SSLEngineResult: "
-                                    + wrapHelloResult.toString());
-                        }
-
-                        logger.trace("{} Handshake response after wrapping: {}", this, wrapHelloResult);
-
-                        final ByteBuffer readableStreamOut = streamOutManager.prepareForRead(1);
-                        final int bytesToSend = readableStreamOut.remaining();
-                        writeFully(readableStreamOut);
-                        logger.trace("{} Sent {} bytes of wrapped data for handshake", this, bytesToSend);
-
-                        streamOutManager.clear();
-                    }
-                    continue;
-                    case NEED_UNWRAP: {
-                        final ByteBuffer readableDataIn = streamInManager.prepareForRead(0);
-                        final ByteBuffer appData = appDataManager.prepareForWrite(engine.getSession().getApplicationBufferSize());
-
-                        // Read handshake response from other side
-                        logger.trace("{} Unwrapping: {} to {}", new Object[]{this, readableDataIn, appData});
-                        SSLEngineResult handshakeResponseResult = engine.unwrap(readableDataIn, appData);
-                        logger.trace("{} Handshake response after unwrapping: {}", this, handshakeResponseResult);
-
-                        if (handshakeResponseResult.getStatus() == Status.BUFFER_UNDERFLOW) {
-                            final ByteBuffer writableDataIn = streamInManager.prepareForWrite(engine.getSession().getPacketBufferSize());
-                            final int bytesRead = readData(writableDataIn);
-                            if (bytesRead > 0) {
-                                logger.trace("{} Read {} bytes for handshake", this, bytesRead);
-                            }
-
-                            if (bytesRead < 0) {
-                                throw new SSLHandshakeException("Reached End-of-File marker while performing handshake");
-                            }
-                        } else if (handshakeResponseResult.getStatus() == Status.CLOSED) {
-                            throw new IOException("Channel was closed by peer during handshake");
-                        } else {
-                            streamInManager.compact();
-                            appDataManager.clear();
-                        }
-                    }
-                    break;
-                    case NEED_TASK:
-                        performTasks();
-                        continue;
-                    case NOT_HANDSHAKING:
-                        return;
-                }
-            }
-        } finally {
-            handshaking = false;
-        }
-    }
-
-    private void performTasks() {
-        Runnable runnable;
-        while ((runnable = engine.getDelegatedTask()) != null) {
-            runnable.run();
-        }
-    }
-
-    private void closeQuietly(final Closeable closeable) {
-        try {
-            closeable.close();
-        } catch (final Exception e) {
-        }
-    }
-
-    private int readData(final ByteBuffer dest) throws IOException {
-        final long startTime = System.currentTimeMillis();
-
-        while (true) {
-            if (interrupted) {
-                throw new TransmissionDisabledException();
-            }
-
-            if (dest.remaining() == 0) {
-                return 0;
-            }
-
-            final int readCount = channel.read(dest);
-
-            if (readCount == 0) {
-                if (System.currentTimeMillis() > startTime + timeoutMillis) {
-                    throw new SocketTimeoutException("Timed out reading from socket connected to " + hostname + ":" + port);
-                }
-                try {
-                    TimeUnit.NANOSECONDS.sleep(BUFFER_FULL_EMPTY_WAIT_NANOS);
-                } catch (InterruptedException e) {
-                    close();
-                    Thread.currentThread().interrupt(); // set the interrupt status
-                    throw new ClosedByInterruptException();
-                }
-
-                continue;
-            }
-
-            logger.trace("{} Read {} bytes", this, readCount);
-            return readCount;
-        }
-    }
-
-    private Status encryptAndWriteFully(final BufferStateManager src) throws IOException {
-        SSLEngineResult result = null;
-
-        final ByteBuffer buff = src.prepareForRead(0);
-        final ByteBuffer outBuff = streamOutManager.prepareForWrite(engine.getSession().getApplicationBufferSize());
-
-        logger.trace("{} Encrypting {} bytes", this, buff.remaining());
-        while (buff.remaining() > 0) {
-            result = engine.wrap(buff, outBuff);
-            if (result.getStatus() == Status.OK) {
-                final ByteBuffer readableOutBuff = streamOutManager.prepareForRead(0);
-                writeFully(readableOutBuff);
-                streamOutManager.clear();
-            } else {
-                return result.getStatus();
-            }
-        }
-
-        return result.getStatus();
-    }
-
-    private void writeFully(final ByteBuffer src) throws IOException {
-        long lastByteWrittenTime = System.currentTimeMillis();
-
-        int bytesWritten = 0;
-        while (src.hasRemaining()) {
-            if (interrupted) {
-                throw new TransmissionDisabledException();
-            }
-
-            final int written = channel.write(src);
-            bytesWritten += written;
-            final long now = System.currentTimeMillis();
-            if (written > 0) {
-                lastByteWrittenTime = now;
-            } else {
-                if (now > lastByteWrittenTime + timeoutMillis) {
-                    throw new SocketTimeoutException("Timed out writing to socket connected to " + hostname + ":" + port);
-                }
-                try {
-                    TimeUnit.NANOSECONDS.sleep(BUFFER_FULL_EMPTY_WAIT_NANOS);
-                } catch (final InterruptedException e) {
-                    close();
-                    Thread.currentThread().interrupt(); // set the interrupt status
-                    throw new ClosedByInterruptException();
-                }
-            }
-        }
-
-        logger.trace("{} Wrote {} bytes", this, bytesWritten);
-    }
-
-    public boolean isClosed() {
-        if (closed) {
-            return true;
-        }
-        // need to detect if peer has sent closure handshake...if so the answer is true
-        final ByteBuffer writableInBuffer = streamInManager.prepareForWrite(engine.getSession().getPacketBufferSize());
-        int readCount = 0;
-        try {
-            readCount = channel.read(writableInBuffer);
-        } catch (IOException e) {
-            logger.error("{} Failed to readData due to {}", new Object[]{this, e});
-            if (logger.isDebugEnabled()) {
-                logger.error("", e);
-            }
-            readCount = -1; // treat the condition same as if End of Stream
-        }
-        if (readCount == 0) {
-            return false;
-        }
-        if (readCount > 0) {
-            logger.trace("{} Read {} bytes", this, readCount);
-
-            final ByteBuffer streamInBuffer = streamInManager.prepareForRead(1);
-            final ByteBuffer appDataBuffer = appDataManager.prepareForWrite(engine.getSession().getApplicationBufferSize());
-            try {
-                SSLEngineResult unwrapResponse = engine.unwrap(streamInBuffer, appDataBuffer);
-                logger.trace("{} When checking if closed, (handshake={}) Unwrap response: {}", new Object[]{this, handshaking, unwrapResponse});
-                if (unwrapResponse.getStatus().equals(Status.CLOSED)) {
-                    // Drain the incoming TCP buffer
-                    final ByteBuffer discardBuffer = ByteBuffer.allocate(8192);
-                    int bytesDiscarded = channel.read(discardBuffer);
-                    while (bytesDiscarded > 0) {
-                        discardBuffer.clear();
-                        bytesDiscarded = channel.read(discardBuffer);
-                    }
-                    engine.closeInbound();
-                } else {
-                    streamInManager.compact();
-                    return false;
-                }
-            } catch (IOException e) {
-                logger.error("{} Failed to check if closed due to {}. Closing channel.", new Object[]{this, e});
-                if (logger.isDebugEnabled()) {
-                    logger.error("", e);
-                }
-            }
-        }
-        // either readCount is -1, indicating an end of stream, or the peer sent a closure handshake
-        // so go ahead and close down the channel
-        closeQuietly(channel.socket());
-        closeQuietly(channel);
-        closed = true;
-        return true;
-    }
-
-    @Override
-    public void close() throws IOException {
-        logger.debug("{} Closing Connection", this);
-        if (channel == null) {
-            return;
-        }
-
-        if (closed) {
-            return;
-        }
-
-        try {
-            engine.closeOutbound();
-
-            final byte[] emptyMessage = new byte[0];
-
-            final ByteBuffer appDataOut = ByteBuffer.wrap(emptyMessage);
-            final ByteBuffer outboundBuffer = streamOutManager.prepareForWrite(engine.getSession().getApplicationBufferSize());
-            final SSLEngineResult handshakeResult = engine.wrap(appDataOut, outboundBuffer);
-
-            if (handshakeResult.getStatus() != Status.CLOSED) {
-                throw new IOException("Invalid close state - will not send network data");
-            }
-
-            final ByteBuffer readableStreamOut = streamOutManager.prepareForRead(1);
-            writeFully(readableStreamOut);
-        } finally {
-            // Drain the incoming TCP buffer
-            final ByteBuffer discardBuffer = ByteBuffer.allocate(8192);
-            try {
-                int bytesDiscarded = channel.read(discardBuffer);
-                while (bytesDiscarded > 0) {
-                    discardBuffer.clear();
-                    bytesDiscarded = channel.read(discardBuffer);
-                }
-            } catch (Exception e) {
-            }
-
-            closeQuietly(channel.socket());
-            closeQuietly(channel);
-            closed = true;
-        }
-    }
-
-    private int copyFromAppDataBuffer(final byte[] buffer, final int offset, final int len) {
-        // If any data already exists in the application data buffer, copy it to the buffer.
-        final ByteBuffer appDataBuffer = appDataManager.prepareForRead(1);
-
-        final int appDataRemaining = appDataBuffer.remaining();
-        if (appDataRemaining > 0) {
-            final int bytesToCopy = Math.min(len, appDataBuffer.remaining());
-            appDataBuffer.get(buffer, offset, bytesToCopy);
-
-            final int bytesCopied = appDataRemaining - appDataBuffer.remaining();
-            logger.trace("{} Copied {} ({}) bytes from unencrypted application buffer to user space",
-                    new Object[]{this, bytesToCopy, bytesCopied});
-            return bytesCopied;
-        }
-        return 0;
-    }
-
-    public int available() throws IOException {
-        ByteBuffer appDataBuffer = appDataManager.prepareForRead(1);
-        ByteBuffer streamDataBuffer = streamInManager.prepareForRead(1);
-        final int buffered = appDataBuffer.remaining() + streamDataBuffer.remaining();
-        if (buffered > 0) {
-            return buffered;
-        }
-
-        final boolean wasAbleToRead = isDataAvailable();
-        if (!wasAbleToRead) {
-            return 0;
-        }
-
-        appDataBuffer = appDataManager.prepareForRead(1);
-        streamDataBuffer = streamInManager.prepareForRead(1);
-        return appDataBuffer.remaining() + streamDataBuffer.remaining();
-    }
-
-    public boolean isDataAvailable() throws IOException {
-        final ByteBuffer appDataBuffer = appDataManager.prepareForRead(1);
-        final ByteBuffer streamDataBuffer = streamInManager.prepareForRead(1);
-
-        if (appDataBuffer.remaining() > 0 || streamDataBuffer.remaining() > 0) {
-            return true;
-        }
-
-        final ByteBuffer writableBuffer = streamInManager.prepareForWrite(engine.getSession().getPacketBufferSize());
-        final int bytesRead = channel.read(writableBuffer);
-        return (bytesRead > 0);
-    }
-
-    public int read() throws IOException {
-        final int bytesRead = read(oneByteBuffer);
-        if (bytesRead == -1) {
-            return -1;
-        }
-        return oneByteBuffer[0] & 0xFF;
-    }
-
-    public int read(final byte[] buffer) throws IOException {
-        return read(buffer, 0, buffer.length);
-    }
-
-    public int read(final byte[] buffer, final int offset, final int len) throws IOException {
-        logger.debug("{} Reading up to {} bytes of data", this, len);
-
-        if (!connected) {
-            connect();
-        }
-
-        int copied = copyFromAppDataBuffer(buffer, offset, len);
-        if (copied > 0) {
-            return copied;
-        }
-
-        appDataManager.clear();
-
-        while (true) {
-            // prepare buffers and call unwrap
-            final ByteBuffer streamInBuffer = streamInManager.prepareForRead(1);
-            SSLEngineResult unwrapResponse = null;
-            final ByteBuffer appDataBuffer = appDataManager.prepareForWrite(engine.getSession().getApplicationBufferSize());
-            unwrapResponse = engine.unwrap(streamInBuffer, appDataBuffer);
-            logger.trace("{} When reading data, (handshake={}) Unwrap response: {}", new Object[]{this, handshaking, unwrapResponse});
-
-            switch (unwrapResponse.getStatus()) {
-                case BUFFER_OVERFLOW:
-                    throw new SSLHandshakeException("Buffer Overflow, which is not allowed to happen from an unwrap");
-                case BUFFER_UNDERFLOW: {
-//                appDataManager.prepareForRead(engine.getSession().getApplicationBufferSize());
-
-                    final ByteBuffer writableInBuffer = streamInManager.prepareForWrite(engine.getSession().getPacketBufferSize());
-                    final int bytesRead = readData(writableInBuffer);
-                    if (bytesRead < 0) {
-                        return -1;
-                    }
-
-                    continue;
-                }
-                case CLOSED:
-                    throw new IOException("Channel is closed");
-                case OK: {
-                    copied = copyFromAppDataBuffer(buffer, offset, len);
-                    if (copied == 0) {
-                        throw new IOException("Failed to decrypt data");
-                    }
-                    streamInManager.compact();
-                    return copied;
-                }
-            }
-        }
-    }
-
-    public void write(final int data) throws IOException {
-        write(new byte[]{(byte) data}, 0, 1);
-    }
-
-    public void write(final byte[] data) throws IOException {
-        write(data, 0, data.length);
-    }
-
-    public void write(final byte[] data, final int offset, final int len) throws IOException {
-        logger.debug("{} Writing {} bytes of data", this, len);
-
-        if (!connected) {
-            connect();
-        }
-
-        int iterations = len / MAX_WRITE_SIZE;
-        if (len % MAX_WRITE_SIZE > 0) {
-            iterations++;
-        }
-
-        for (int i = 0; i < iterations; i++) {
-            streamOutManager.clear();
-            final int itrOffset = offset + i * MAX_WRITE_SIZE;
-            final int itrLen = Math.min(len - itrOffset, MAX_WRITE_SIZE);
-            final ByteBuffer byteBuffer = ByteBuffer.wrap(data, itrOffset, itrLen);
-
-            final BufferStateManager buffMan = new BufferStateManager(byteBuffer, Direction.READ);
-            final Status status = encryptAndWriteFully(buffMan);
-            switch (status) {
-                case BUFFER_OVERFLOW:
-                    streamOutManager.ensureSize(engine.getSession().getPacketBufferSize());
-                    appDataManager.ensureSize(engine.getSession().getApplicationBufferSize());
-                    continue;
-                case OK:
-                    continue;
-                case CLOSED:
-                    throw new IOException("Channel is closed");
-                case BUFFER_UNDERFLOW:
-                    throw new AssertionError("Got Buffer Underflow but should not have...");
-            }
-        }
-    }
-
-    public void interrupt() {
-        this.interrupted = true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInputStream.java
deleted file mode 100644
index 154bd08..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInputStream.java
+++ /dev/null
@@ -1,62 +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.remote.io.socket.ssl;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-public class SSLSocketChannelInputStream extends InputStream {
-
-    private final SSLSocketChannel channel;
-
-    public SSLSocketChannelInputStream(final SSLSocketChannel channel) {
-        this.channel = channel;
-    }
-
-    @Override
-    public int read() throws IOException {
-        return channel.read();
-    }
-
-    @Override
-    public int read(final byte[] b) throws IOException {
-        return channel.read(b);
-    }
-
-    @Override
-    public int read(final byte[] b, final int off, final int len) throws IOException {
-        return channel.read(b, off, len);
-    }
-
-    /**
-     * Closes the underlying SSLSocketChannel, which will also close the
-     * OutputStream and connection
-     */
-    @Override
-    public void close() throws IOException {
-        channel.close();
-    }
-
-    @Override
-    public int available() throws IOException {
-        return channel.available();
-    }
-
-    public boolean isDataAvailable() throws IOException {
-        return available() > 0;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutputStream.java
deleted file mode 100644
index ce4e420..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutputStream.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.remote.io.socket.ssl;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-public class SSLSocketChannelOutputStream extends OutputStream {
-
-    private final SSLSocketChannel channel;
-
-    public SSLSocketChannelOutputStream(final SSLSocketChannel channel) {
-        this.channel = channel;
-    }
-
-    @Override
-    public void write(final int b) throws IOException {
-        channel.write(b);
-    }
-
-    @Override
-    public void write(byte[] b) throws IOException {
-        channel.write(b);
-    }
-
-    @Override
-    public void write(byte[] b, int off, int len) throws IOException {
-        channel.write(b, off, len);
-    }
-
-    /**
-     * Closes the underlying SSLSocketChannel, which also will close the
-     * InputStream and the connection
-     */
-    @Override
-    public void close() throws IOException {
-        channel.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/BufferedInputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/BufferedInputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/BufferedInputStream.java
deleted file mode 100644
index aaf37ea..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/BufferedInputStream.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.stream.io;
-
-import java.io.InputStream;
-
-/**
- * This class is a slight modification of the BufferedInputStream in the java.io
- * package. The modification is that this implementation does not provide
- * synchronization on method calls, which means that this class is not suitable
- * for use by multiple threads. However, the absence of these synchronized
- * blocks results in potentially much better performance.
- */
-public class BufferedInputStream extends java.io.BufferedInputStream {
-
-    public BufferedInputStream(final InputStream in) {
-        super(in);
-    }
-
-    public BufferedInputStream(final InputStream in, final int size) {
-        super(in, size);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/BufferedOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/BufferedOutputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/BufferedOutputStream.java
deleted file mode 100644
index eadfcab..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/BufferedOutputStream.java
+++ /dev/null
@@ -1,140 +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.stream.io;
-
-import java.io.FilterOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-
-/**
- * This class is a slight modification of the
- * {@link java.io.BufferedOutputStream} class. This implementation differs in
- * that it does not mark methods as synchronized. This means that this class is
- * not suitable for writing by multiple concurrent threads. However, the removal
- * of the synchronized keyword results in potentially much better performance.
- */
-public class BufferedOutputStream extends FilterOutputStream {
-
-    /**
-     * The internal buffer where data is stored.
-     */
-    protected byte buf[];
-
-    /**
-     * The number of valid bytes in the buffer. This value is always in the
-     * range <tt>0</tt> through <tt>buf.length</tt>; elements
-     * <tt>buf[0]</tt> through <tt>buf[count-1]</tt> contain valid byte data.
-     */
-    protected int count;
-
-    /**
-     * Creates a new buffered output stream to write data to the specified
-     * underlying output stream.
-     *
-     * @param out the underlying output stream.
-     */
-    public BufferedOutputStream(OutputStream out) {
-        this(out, 8192);
-    }
-
-    /**
-     * Creates a new buffered output stream to write data to the specified
-     * underlying output stream with the specified buffer size.
-     *
-     * @param out the underlying output stream.
-     * @param size the buffer size.
-     * @exception IllegalArgumentException if size &lt;= 0.
-     */
-    public BufferedOutputStream(OutputStream out, int size) {
-        super(out);
-        if (size <= 0) {
-            throw new IllegalArgumentException("Buffer size <= 0");
-        }
-        buf = new byte[size];
-    }
-
-    /**
-     * Flush the internal buffer
-     */
-    private void flushBuffer() throws IOException {
-        if (count > 0) {
-            out.write(buf, 0, count);
-            count = 0;
-        }
-    }
-
-    /**
-     * Writes the specified byte to this buffered output stream.
-     *
-     * @param b the byte to be written.
-     * @exception IOException if an I/O error occurs.
-     */
-    @Override
-    public void write(int b) throws IOException {
-        if (count >= buf.length) {
-            flushBuffer();
-        }
-        buf[count++] = (byte) b;
-    }
-
-    /**
-     * Writes <code>len</code> bytes from the specified byte array starting at
-     * offset <code>off</code> to this buffered output stream.
-     *
-     * <p>
-     * Ordinarily this method stores bytes from the given array into this
-     * stream's buffer, flushing the buffer to the underlying output stream as
-     * needed. If the requested length is at least as large as this stream's
-     * buffer, however, then this method will flush the buffer and write the
-     * bytes directly to the underlying output stream. Thus redundant
-     * <code>BufferedOutputStream</code>s will not copy data unnecessarily.
-     *
-     * @param b the data.
-     * @param off the start offset in the data.
-     * @param len the number of bytes to write.
-     * @exception IOException if an I/O error occurs.
-     */
-    @Override
-    public void write(byte b[], int off, int len) throws IOException {
-        if (len >= buf.length) {
-            /* If the request length exceeds the size of the output buffer,
-             flush the output buffer and then write the data directly.
-             In this way buffered streams will cascade harmlessly. */
-            flushBuffer();
-            out.write(b, off, len);
-            return;
-        }
-        if (len >= buf.length - count) {
-            flushBuffer();
-        }
-        System.arraycopy(b, off, buf, count, len);
-        count += len;
-    }
-
-    /**
-     * Flushes this buffered output stream. This forces any buffered output
-     * bytes to be written out to the underlying output stream.
-     *
-     * @exception IOException if an I/O error occurs.
-     * @see java.io.FilterOutputStream#out
-     */
-    @Override
-    public void flush() throws IOException {
-        flushBuffer();
-        out.flush();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteArrayInputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteArrayInputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteArrayInputStream.java
deleted file mode 100644
index 284cd54..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteArrayInputStream.java
+++ /dev/null
@@ -1,250 +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.stream.io;
-
-import java.io.InputStream;
-
-/**
- * This class performs the same function as java.io.ByteArrayInputStream but
- * does not mark its methods as synchronized
- */
-public class ByteArrayInputStream extends InputStream {
-
-    /**
-     * An array of bytes that was provided by the creator of the stream.
-     * Elements <code>buf[0]</code> through <code>buf[count-1]</code> are the
-     * only bytes that can ever be read from the stream; element
-     * <code>buf[pos]</code> is the next byte to be read.
-     */
-    protected byte buf[];
-
-    /**
-     * The index of the next character to read from the input stream buffer.
-     * This value should always be nonnegative and not larger than the value of
-     * <code>count</code>. The next byte to be read from the input stream buffer
-     * will be <code>buf[pos]</code>.
-     */
-    protected int pos;
-
-    /**
-     * The currently marked position in the stream. ByteArrayInputStream objects
-     * are marked at position zero by default when constructed. They may be
-     * marked at another position within the buffer by the <code>mark()</code>
-     * method. The current buffer position is set to this point by the
-     * <code>reset()</code> method.
-     * <p>
-     * If no mark has been set, then the value of mark is the offset passed to
-     * the constructor (or 0 if the offset was not supplied).
-     *
-     * @since JDK1.1
-     */
-    protected int mark = 0;
-
-    /**
-     * The index one greater than the last valid character in the input stream
-     * buffer. This value should always be nonnegative and not larger than the
-     * length of <code>buf</code>. It is one greater than the position of the
-     * last byte within <code>buf</code> that can ever be read from the input
-     * stream buffer.
-     */
-    protected int count;
-
-    /**
-     * Creates a <code>ByteArrayInputStream</code> so that it uses
-     * <code>buf</code> as its buffer array. The buffer array is not copied. The
-     * initial value of <code>pos</code> is <code>0</code> and the initial value
-     * of  <code>count</code> is the length of <code>buf</code>.
-     *
-     * @param buf the input buffer.
-     */
-    public ByteArrayInputStream(byte buf[]) {
-        this.buf = buf;
-        this.pos = 0;
-        this.count = buf.length;
-    }
-
-    /**
-     * Creates <code>ByteArrayInputStream</code> that uses <code>buf</code> as
-     * its buffer array. The initial value of <code>pos</code> is
-     * <code>offset</code> and the initial value of <code>count</code> is the
-     * minimum of <code>offset+length</code> and <code>buf.length</code>. The
-     * buffer array is not copied. The buffer's mark is set to the specified
-     * offset.
-     *
-     * @param buf the input buffer.
-     * @param offset the offset in the buffer of the first byte to read.
-     * @param length the maximum number of bytes to read from the buffer.
-     */
-    public ByteArrayInputStream(byte buf[], int offset, int length) {
-        this.buf = buf;
-        this.pos = offset;
-        this.count = Math.min(offset + length, buf.length);
-        this.mark = offset;
-    }
-
-    /**
-     * Reads the next byte of data from this input stream. The value byte is
-     * returned as an <code>int</code> in the range <code>0</code> to
-     * <code>255</code>. If no byte is available because the end of the stream
-     * has been reached, the value <code>-1</code> is returned.
-     * <p>
-     * This <code>read</code> method cannot block.
-     *
-     * @return the next byte of data, or <code>-1</code> if the end of the
-     * stream has been reached.
-     */
-    @Override
-    public int read() {
-        return (pos < count) ? (buf[pos++] & 0xff) : -1;
-    }
-
-    /**
-     * Reads up to <code>len</code> bytes of data into an array of bytes from
-     * this input stream. If <code>pos</code> equals <code>count</code>, then
-     * <code>-1</code> is returned to indicate end of file. Otherwise, the
-     * number <code>k</code> of bytes read is equal to the smaller of
-     * <code>len</code> and <code>count-pos</code>. If <code>k</code> is
-     * positive, then bytes <code>buf[pos]</code> through
-     * <code>buf[pos+k-1]</code> are copied into <code>b[off]</code> through
-     * <code>b[off+k-1]</code> in the manner performed by
-     * <code>System.arraycopy</code>. The value <code>k</code> is added into
-     * <code>pos</code> and <code>k</code> is returned.
-     * <p>
-     * This <code>read</code> method cannot block.
-     *
-     * @param b the buffer into which the data is read.
-     * @param off the start offset in the destination array <code>b</code>
-     * @param len the maximum number of bytes read.
-     * @return the total number of bytes read into the buffer, or
-     * <code>-1</code> if there is no more data because the end of the stream
-     * has been reached.
-     * @exception NullPointerException If <code>b</code> is <code>null</code>.
-     * @exception IndexOutOfBoundsException If <code>off</code> is negative,
-     * <code>len</code> is negative, or <code>len</code> is greater than
-     * <code>b.length - off</code>
-     */
-    @Override
-    public int read(byte b[], int off, int len) {
-        if (b == null) {
-            throw new NullPointerException();
-        } else if (off < 0 || len < 0 || len > b.length - off) {
-            throw new IndexOutOfBoundsException();
-        }
-
-        if (pos >= count) {
-            return -1;
-        }
-
-        int avail = count - pos;
-        if (len > avail) {
-            len = avail;
-        }
-        if (len <= 0) {
-            return 0;
-        }
-        System.arraycopy(buf, pos, b, off, len);
-        pos += len;
-        return len;
-    }
-
-    /**
-     * Skips <code>n</code> bytes of input from this input stream. Fewer bytes
-     * might be skipped if the end of the input stream is reached. The actual
-     * number <code>k</code> of bytes to be skipped is equal to the smaller of
-     * <code>n</code> and  <code>count-pos</code>. The value <code>k</code> is
-     * added into <code>pos</code> and <code>k</code> is returned.
-     *
-     * @param n the number of bytes to be skipped.
-     * @return the actual number of bytes skipped.
-     */
-    @Override
-    public long skip(long n) {
-        long k = count - pos;
-        if (n < k) {
-            k = n < 0 ? 0 : n;
-        }
-
-        pos += k;
-        return k;
-    }
-
-    /**
-     * Returns the number of remaining bytes that can be read (or skipped over)
-     * from this input stream.
-     * <p>
-     * The value returned is <code>count&nbsp;- pos</code>, which is the number
-     * of bytes remaining to be read from the input buffer.
-     *
-     * @return the number of remaining bytes that can be read (or skipped over)
-     * from this input stream without blocking.
-     */
-    @Override
-    public int available() {
-        return count - pos;
-    }
-
-    /**
-     * Tests if this <code>InputStream</code> supports mark/reset. The
-     * <code>markSupported</code> method of <code>ByteArrayInputStream</code>
-     * always returns <code>true</code>.
-     *
-     * @since JDK1.1
-     */
-    @Override
-    public boolean markSupported() {
-        return true;
-    }
-
-    /**
-     * Set the current marked position in the stream. ByteArrayInputStream
-     * objects are marked at position zero by default when constructed. They may
-     * be marked at another position within the buffer by this method.
-     * <p>
-     * If no mark has been set, then the value of the mark is the offset passed
-     * to the constructor (or 0 if the offset was not supplied).
-     *
-     * <p>
-     * Note: The <code>readAheadLimit</code> for this class has no meaning.
-     *
-     * @since JDK1.1
-     */
-    @Override
-    public void mark(int readAheadLimit) {
-        mark = pos;
-    }
-
-    /**
-     * Resets the buffer to the marked position. The marked position is 0 unless
-     * another position was marked or an offset was specified in the
-     * constructor.
-     */
-    @Override
-    public void reset() {
-        pos = mark;
-    }
-
-    /**
-     * Closing a <tt>ByteArrayInputStream</tt> has no effect. The methods in
-     * this class can be called after the stream has been closed without
-     * generating an <tt>IOException</tt>.
-     * <p>
-     */
-    @Override
-    public void close() {
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteArrayOutputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteArrayOutputStream.java
deleted file mode 100644
index 459563b..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteArrayOutputStream.java
+++ /dev/null
@@ -1,250 +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.stream.io;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.UnsupportedEncodingException;
-import java.util.Arrays;
-
-/**
- * This class provides a more efficient implementation of the
- * java.io.ByteArrayOutputStream. The efficiency is gained in two ways:
- * <ul>
- * <li>The write methods are not synchronized</li>
- * <li>The class provides {@link #getUnderlyingBuffer()} and
- * {@link #getBufferLength()}, which can be used to access the underlying byte
- * array directly, rather than the System.arraycopy that {@link #toByteArray()}
- * uses
- * </ul>
- *
- */
-public class ByteArrayOutputStream extends OutputStream {
-
-    /**
-     * The buffer where data is stored.
-     */
-    protected byte buf[];
-
-    /**
-     * The number of valid bytes in the buffer.
-     */
-    protected int count;
-
-    /**
-     * Creates a new byte array output stream. The buffer capacity is initially
-     * 32 bytes, though its size increases if necessary.
-     */
-    public ByteArrayOutputStream() {
-        this(32);
-    }
-
-    /**
-     * Creates a new byte array output stream, with a buffer capacity of the
-     * specified size, in bytes.
-     *
-     * @param size the initial size.
-     * @exception IllegalArgumentException if size is negative.
-     */
-    public ByteArrayOutputStream(int size) {
-        if (size < 0) {
-            throw new IllegalArgumentException("Negative initial size: "
-                    + size);
-        }
-        buf = new byte[size];
-    }
-
-    /**
-     * Increases the capacity if necessary to ensure that it can hold at least
-     * the number of elements specified by the minimum capacity argument.
-     *
-     * @param minCapacity the desired minimum capacity
-     * @throws OutOfMemoryError if {@code minCapacity < 0}. This is interpreted
-     * as a request for the unsatisfiably large capacity
-     * {@code (long) Integer.MAX_VALUE + (minCapacity - Integer.MAX_VALUE)}.
-     */
-    private void ensureCapacity(int minCapacity) {
-        // overflow-conscious code
-        if (minCapacity - buf.length > 0) {
-            grow(minCapacity);
-        }
-    }
-
-    /**
-     * Increases the capacity to ensure that it can hold at least the number of
-     * elements specified by the minimum capacity argument.
-     *
-     * @param minCapacity the desired minimum capacity
-     */
-    private void grow(int minCapacity) {
-        // overflow-conscious code
-        int oldCapacity = buf.length;
-        int newCapacity = oldCapacity << 1;
-        if (newCapacity - minCapacity < 0) {
-            newCapacity = minCapacity;
-        }
-        if (newCapacity < 0) {
-            if (minCapacity < 0) // overflow
-            {
-                throw new OutOfMemoryError();
-            }
-            newCapacity = Integer.MAX_VALUE;
-        }
-        buf = Arrays.copyOf(buf, newCapacity);
-    }
-
-    /**
-     * Writes the specified byte to this byte array output stream.
-     *
-     * @param b the byte to be written.
-     */
-    @Override
-    public void write(int b) {
-        ensureCapacity(count + 1);
-        buf[count] = (byte) b;
-        count += 1;
-    }
-
-    /**
-     * Writes <code>len</code> bytes from the specified byte array starting at
-     * offset <code>off</code> to this byte array output stream.
-     *
-     * @param b the data.
-     * @param off the start offset in the data.
-     * @param len the number of bytes to write.
-     */
-    @Override
-    public void write(byte b[], int off, int len) {
-        if ((off < 0) || (off > b.length) || (len < 0)
-                || ((off + len) - b.length > 0)) {
-            throw new IndexOutOfBoundsException();
-        }
-        ensureCapacity(count + len);
-        System.arraycopy(b, off, buf, count, len);
-        count += len;
-    }
-
-    /**
-     * Writes the complete contents of this byte array output stream to the
-     * specified output stream argument, as if by calling the output stream's
-     * write method using <code>out.write(buf, 0, count)</code>.
-     *
-     * @param out the output stream to which to write the data.
-     * @exception IOException if an I/O error occurs.
-     */
-    public void writeTo(OutputStream out) throws IOException {
-        out.write(buf, 0, count);
-    }
-
-    /**
-     * Resets the <code>count</code> field of this byte array output stream to
-     * zero, so that all currently accumulated output in the output stream is
-     * discarded. The output stream can be used again, reusing the already
-     * allocated buffer space.
-     *
-     * @see java.io.ByteArrayInputStream#count
-     */
-    public void reset() {
-        count = 0;
-    }
-
-    /**
-     * Creates a newly allocated byte array. Its size is the current size of
-     * this output stream and the valid contents of the buffer have been copied
-     * into it.
-     *
-     * @return the current contents of this output stream, as a byte array.
-     * @see java.io.ByteArrayOutputStream#size()
-     */
-    public byte toByteArray   () 
-        [] {
-        return Arrays.copyOf(buf, count);
-    }
-
-    /**
-     * Returns the current size of the buffer.
-     *
-     * @return the value of the <code>count</code> field, which is the number of
-     * valid bytes in this output stream.
-     * @see java.io.ByteArrayOutputStream#count
-     */
-    public int size() {
-        return count;
-    }
-
-    /**
-     * Converts the buffer's contents into a string decoding bytes using the
-     * platform's default character set. The length of the new <tt>String</tt>
-     * is a function of the character set, and hence may not be equal to the
-     * size of the buffer.
-     *
-     * <p>
-     * This method always replaces malformed-input and unmappable-character
-     * sequences with the default replacement string for the platform's default
-     * character set. The {@linkplain java.nio.charset.CharsetDecoder} class
-     * should be used when more control over the decoding process is required.
-     *
-     * @return String decoded from the buffer's contents.
-     * @since JDK1.1
-     */
-    @Override
-    public String toString() {
-        return new String(buf, 0, count);
-    }
-
-    /**
-     * Converts the buffer's contents into a string by decoding the bytes using
-     * the specified {@link java.nio.charset.Charset charsetName}. The length of
-     * the new <tt>String</tt> is a function of the charset, and hence may not
-     * be equal to the length of the byte array.
-     *
-     * <p>
-     * This method always replaces malformed-input and unmappable-character
-     * sequences with this charset's default replacement string. The {@link
-     * java.nio.charset.CharsetDecoder} class should be used when more control
-     * over the decoding process is required.
-     *
-     * @param charsetName the name of a supported
-     *              {@linkplain java.nio.charset.Charset <code>charset</code>}
-     * @return String decoded from the buffer's contents.
-     * @exception UnsupportedEncodingException If the named charset is not
-     * supported
-     * @since JDK1.1
-     */
-    public String toString(String charsetName) throws UnsupportedEncodingException {
-        return new String(buf, 0, count, charsetName);
-    }
-
-    /**
-     * Closing a <tt>ByteArrayOutputStream</tt> has no effect. The methods in
-     * this class can be called after the stream has been closed without
-     * generating an <tt>IOException</tt>.
-     * <p>
-     *
-     */
-    @Override
-    public void close() {
-    }
-
-    public byte[] getUnderlyingBuffer() {
-        return buf;
-    }
-
-    public int getBufferLength() {
-        return count;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingInputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingInputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingInputStream.java
deleted file mode 100644
index 8294af3..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingInputStream.java
+++ /dev/null
@@ -1,104 +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.stream.io;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-public class ByteCountingInputStream extends InputStream {
-
-    private final InputStream in;
-    private long bytesRead = 0L;
-    private long bytesSkipped = 0L;
-
-    private long bytesSinceMark = 0L;
-
-    public ByteCountingInputStream(final InputStream in) {
-        this.in = in;
-    }
-
-    @Override
-    public int read() throws IOException {
-        final int fromSuper = in.read();
-        if (fromSuper >= 0) {
-            bytesRead++;
-            bytesSinceMark++;
-        }
-        return fromSuper;
-    }
-
-    @Override
-    public int read(byte[] b, int off, int len) throws IOException {
-        final int fromSuper = in.read(b, off, len);
-        if (fromSuper >= 0) {
-            bytesRead += fromSuper;
-            bytesSinceMark += fromSuper;
-        }
-
-        return fromSuper;
-    }
-
-    @Override
-    public int read(byte[] b) throws IOException {
-        return read(b, 0, b.length);
-    }
-
-    @Override
-    public long skip(final long n) throws IOException {
-        final long skipped = in.skip(n);
-        if (skipped >= 0) {
-            bytesSkipped += skipped;
-            bytesSinceMark += skipped;
-        }
-        return skipped;
-    }
-
-    public long getBytesRead() {
-        return bytesRead;
-    }
-
-    public long getBytesSkipped() {
-        return bytesSkipped;
-    }
-
-    public long getBytesConsumed() {
-        return getBytesRead() + getBytesSkipped();
-    }
-
-    @Override
-    public void mark(final int readlimit) {
-        in.mark(readlimit);
-
-        bytesSinceMark = 0L;
-    }
-
-    @Override
-    public boolean markSupported() {
-        return in.markSupported();
-    }
-
-    @Override
-    public void reset() throws IOException {
-        in.reset();
-        bytesRead -= bytesSinceMark;
-    }
-
-    @Override
-    public void close() throws IOException {
-        in.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingOutputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingOutputStream.java
deleted file mode 100644
index d8e1a42..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingOutputStream.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.stream.io;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-public class ByteCountingOutputStream extends OutputStream {
-
-    private final OutputStream out;
-    private long bytesWritten = 0L;
-
-    public ByteCountingOutputStream(final OutputStream out) {
-        this.out = out;
-    }
-
-    @Override
-    public void write(int b) throws IOException {
-        out.write(b);
-        bytesWritten++;
-    }
-
-    @Override
-    public void write(byte[] b) throws IOException {
-        write(b, 0, b.length);
-    }
-
-    ;
-    
-    @Override
-    public void write(byte[] b, int off, int len) throws IOException {
-        out.write(b, off, len);
-        bytesWritten += len;
-    }
-
-    public long getBytesWritten() {
-        return bytesWritten;
-    }
-
-    @Override
-    public void flush() throws IOException {
-        out.flush();
-    }
-
-    @Override
-    public void close() throws IOException {
-        out.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/DataOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/DataOutputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/DataOutputStream.java
deleted file mode 100644
index 1dd90f5..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/DataOutputStream.java
+++ /dev/null
@@ -1,417 +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.stream.io;
-
-import java.io.DataOutput;
-import java.io.FilterOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.UTFDataFormatException;
-
-/**
- * This class is different from java.io.DataOutputStream in that it does
- * synchronize on its methods.
- */
-public class DataOutputStream extends FilterOutputStream implements DataOutput {
-
-    /**
-     * The number of bytes written to the data output stream so far. If this
-     * counter overflows, it will be wrapped to Integer.MAX_VALUE.
-     */
-    protected int written;
-
-    /**
-     * bytearr is initialized on demand by writeUTF
-     */
-    private byte[] bytearr = null;
-
-    /**
-     * Creates a new data output stream to write data to the specified
-     * underlying output stream. The counter <code>written</code> is set to
-     * zero.
-     *
-     * @param out the underlying output stream, to be saved for later use.
-     * @see java.io.FilterOutputStream#out
-     */
-    public DataOutputStream(OutputStream out) {
-        super(out);
-    }
-
-    /**
-     * Increases the written counter by the specified value until it reaches
-     * Integer.MAX_VALUE.
-     */
-    private void incCount(int value) {
-        int temp = written + value;
-        if (temp < 0) {
-            temp = Integer.MAX_VALUE;
-        }
-        written = temp;
-    }
-
-    /**
-     * Writes the specified byte (the low eight bits of the argument
-     * <code>b</code>) to the underlying output stream. If no exception is
-     * thrown, the counter <code>written</code> is incremented by
-     * <code>1</code>.
-     * <p>
-     * Implements the <code>write</code> method of <code>OutputStream</code>.
-     *
-     * @param b the <code>byte</code> to be written.
-     * @exception IOException if an I/O error occurs.
-     * @see java.io.FilterOutputStream#out
-     */
-    @Override
-    public void write(int b) throws IOException {
-        out.write(b);
-        incCount(1);
-    }
-
-    /**
-     * Writes <code>len</code> bytes from the specified byte array starting at
-     * offset <code>off</code> to the underlying output stream. If no exception
-     * is thrown, the counter <code>written</code> is incremented by
-     * <code>len</code>.
-     *
-     * @param b the data.
-     * @param off the start offset in the data.
-     * @param len the number of bytes to write.
-     * @exception IOException if an I/O error occurs.
-     * @see java.io.FilterOutputStream#out
-     */
-    @Override
-    public void write(byte b[], int off, int len) throws IOException {
-        out.write(b, off, len);
-        incCount(len);
-    }
-
-    /**
-     * Flushes this data output stream. This forces any buffered output bytes to
-     * be written out to the stream.
-     * <p>
-     * The <code>flush</code> method of <code>DataOutputStream</code> calls the
-     * <code>flush</code> method of its underlying output stream.
-     *
-     * @exception IOException if an I/O error occurs.
-     * @see java.io.FilterOutputStream#out
-     * @see java.io.OutputStream#flush()
-     */
-    @Override
-    public void flush() throws IOException {
-        out.flush();
-    }
-
-    /**
-     * Writes a <code>boolean</code> to the underlying output stream as a 1-byte
-     * value. The value <code>true</code> is written out as the value
-     * <code>(byte)1</code>; the value <code>false</code> is written out as the
-     * value <code>(byte)0</code>. If no exception is thrown, the counter
-     * <code>written</code> is incremented by <code>1</code>.
-     *
-     * @param v a <code>boolean</code> value to be written.
-     * @exception IOException if an I/O error occurs.
-     * @see java.io.FilterOutputStream#out
-     */
-    @Override
-    public final void writeBoolean(boolean v) throws IOException {
-        out.write(v ? 1 : 0);
-        incCount(1);
-    }
-
-    /**
-     * Writes out a <code>byte</code> to the underlying output stream as a
-     * 1-byte value. If no exception is thrown, the counter <code>written</code>
-     * is incremented by <code>1</code>.
-     *
-     * @param v a <code>byte</code> value to be written.
-     * @exception IOException if an I/O error occurs.
-     * @see java.io.FilterOutputStream#out
-     */
-    @Override
-    public final void writeByte(int v) throws IOException {
-        out.write(v);
-        incCount(1);
-    }
-
-    /**
-     * Writes a <code>short</code> to the underlying output stream as two bytes,
-     * high byte first. If no exception is thrown, the counter
-     * <code>written</code> is incremented by <code>2</code>.
-     *
-     * @param v a <code>short</code> to be written.
-     * @exception IOException if an I/O error occurs.
-     * @see java.io.FilterOutputStream#out
-     */
-    @Override
-    public final void writeShort(int v) throws IOException {
-        out.write((v >>> 8) & 0xFF);
-        out.write((v) & 0xFF);
-        incCount(2);
-    }
-
-    /**
-     * Writes a <code>char</code> to the underlying output stream as a 2-byte
-     * value, high byte first. If no exception is thrown, the counter
-     * <code>written</code> is incremented by <code>2</code>.
-     *
-     * @param v a <code>char</code> value to be written.
-     * @exception IOException if an I/O error occurs.
-     * @see java.io.FilterOutputStream#out
-     */
-    @Override
-    public final void writeChar(int v) throws IOException {
-        out.write((v >>> 8) & 0xFF);
-        out.write((v) & 0xFF);
-        incCount(2);
-    }
-
-    /**
-     * Writes an <code>int</code> to the underlying output stream as four bytes,
-     * high byte first. If no exception is thrown, the counter
-     * <code>written</code> is incremented by <code>4</code>.
-     *
-     * @param v an <code>int</code> to be written.
-     * @exception IOException if an I/O error occurs.
-     * @see java.io.FilterOutputStream#out
-     */
-    @Override
-    public final void writeInt(int v) throws IOException {
-        out.write((v >>> 24) & 0xFF);
-        out.write((v >>> 16) & 0xFF);
-        out.write((v >>> 8) & 0xFF);
-        out.write((v) & 0xFF);
-        incCount(4);
-    }
-
-    private final byte writeBuffer[] = new byte[8];
-
-    /**
-     * Writes a <code>long</code> to the underlying output stream as eight
-     * bytes, high byte first. In no exception is thrown, the counter
-     * <code>written</code> is incremented by <code>8</code>.
-     *
-     * @param v a <code>long</code> to be written.
-     * @exception IOException if an I/O error occurs.
-     * @see java.io.FilterOutputStream#out
-     */
-    @Override
-    public final void writeLong(long v) throws IOException {
-        writeBuffer[0] = (byte) (v >>> 56);
-        writeBuffer[1] = (byte) (v >>> 48);
-        writeBuffer[2] = (byte) (v >>> 40);
-        writeBuffer[3] = (byte) (v >>> 32);
-        writeBuffer[4] = (byte) (v >>> 24);
-        writeBuffer[5] = (byte) (v >>> 16);
-        writeBuffer[6] = (byte) (v >>> 8);
-        writeBuffer[7] = (byte) (v);
-        out.write(writeBuffer, 0, 8);
-        incCount(8);
-    }
-
-    /**
-     * Converts the float argument to an <code>int</code> using the
-     * <code>floatToIntBits</code> method in class <code>Float</code>, and then
-     * writes that <code>int</code> value to the underlying output stream as a
-     * 4-byte quantity, high byte first. If no exception is thrown, the counter
-     * <code>written</code> is incremented by <code>4</code>.
-     *
-     * @param v a <code>float</code> value to be written.
-     * @exception IOException if an I/O error occurs.
-     * @see java.io.FilterOutputStream#out
-     * @see java.lang.Float#floatToIntBits(float)
-     */
-    @Override
-    public final void writeFloat(float v) throws IOException {
-        writeInt(Float.floatToIntBits(v));
-    }
-
-    /**
-     * Converts the double argument to a <code>long</code> using the
-     * <code>doubleToLongBits</code> method in class <code>Double</code>, and
-     * then writes that <code>long</code> value to the underlying output stream
-     * as an 8-byte quantity, high byte first. If no exception is thrown, the
-     * counter <code>written</code> is incremented by <code>8</code>.
-     *
-     * @param v a <code>double</code> value to be written.
-     * @exception IOException if an I/O error occurs.
-     * @see java.io.FilterOutputStream#out
-     * @see java.lang.Double#doubleToLongBits(double)
-     */
-    @Override
-    public final void writeDouble(double v) throws IOException {
-        writeLong(Double.doubleToLongBits(v));
-    }
-
-    /**
-     * Writes out the string to the underlying output stream as a sequence of
-     * bytes. Each character in the string is written out, in sequence, by
-     * discarding its high eight bits. If no exception is thrown, the counter
-     * <code>written</code> is incremented by the length of <code>s</code>.
-     *
-     * @param s a string of bytes to be written.
-     * @exception IOException if an I/O error occurs.
-     * @see java.io.FilterOutputStream#out
-     */
-    @Override
-    public final void writeBytes(String s) throws IOException {
-        int len = s.length();
-        for (int i = 0; i < len; i++) {
-            out.write((byte) s.charAt(i));
-        }
-        incCount(len);
-    }
-
-    /**
-     * Writes a string to the underlying output stream as a sequence of
-     * characters. Each character is written to the data output stream as if by
-     * the <code>writeChar</code> method. If no exception is thrown, the counter
-     * <code>written</code> is incremented by twice the length of
-     * <code>s</code>.
-     *
-     * @param s a <code>String</code> value to be written.
-     * @exception IOException if an I/O error occurs.
-     * @see java.io.DataOutputStream#writeChar(int)
-     * @see java.io.FilterOutputStream#out
-     */
-    @Override
-    public final void writeChars(String s) throws IOException {
-        int len = s.length();
-        for (int i = 0; i < len; i++) {
-            int v = s.charAt(i);
-            out.write((v >>> 8) & 0xFF);
-            out.write((v) & 0xFF);
-        }
-        incCount(len * 2);
-    }
-
-    /**
-     * Writes a string to the underlying output stream using
-     * <a href="DataInput.html#modified-utf-8">modified UTF-8</a>
-     * encoding in a machine-independent manner.
-     * <p>
-     * First, two bytes are written to the output stream as if by the
-     * <code>writeShort</code> method giving the number of bytes to follow. This
-     * value is the number of bytes actually written out, not the length of the
-     * string. Following the length, each character of the string is output, in
-     * sequence, using the modified UTF-8 encoding for the character. If no
-     * exception is thrown, the counter <code>written</code> is incremented by
-     * the total number of bytes written to the output stream. This will be at
-     * least two plus the length of <code>str</code>, and at most two plus
-     * thrice the length of <code>str</code>.
-     *
-     * @param str a string to be written.
-     * @exception IOException if an I/O error occurs.
-     */
-    @Override
-    public final void writeUTF(String str) throws IOException {
-        writeUTF(str, this);
-    }
-
-    /**
-     * Writes a string to the specified DataOutput using
-     * <a href="DataInput.html#modified-utf-8">modified UTF-8</a>
-     * encoding in a machine-independent manner.
-     * <p>
-     * First, two bytes are written to out as if by the <code>writeShort</code>
-     * method giving the number of bytes to follow. This value is the number of
-     * bytes actually written out, not the length of the string. Following the
-     * length, each character of the string is output, in sequence, using the
-     * modified UTF-8 encoding for the character. If no exception is thrown, the
-     * counter <code>written</code> is incremented by the total number of bytes
-     * written to the output stream. This will be at least two plus the length
-     * of <code>str</code>, and at most two plus thrice the length of
-     * <code>str</code>.
-     *
-     * @param str a string to be written.
-     * @param out destination to write to
-     * @return The number of bytes written out.
-     * @exception IOException if an I/O error occurs.
-     */
-    static int writeUTF(String str, DataOutput out) throws IOException {
-        int strlen = str.length();
-        int utflen = 0;
-        int c, count = 0;
-
-        /* use charAt instead of copying String to char array */
-        for (int i = 0; i < strlen; i++) {
-            c = str.charAt(i);
-            if ((c >= 0x0001) && (c <= 0x007F)) {
-                utflen++;
-            } else if (c > 0x07FF) {
-                utflen += 3;
-            } else {
-                utflen += 2;
-            }
-        }
-
-        if (utflen > 65535) {
-            throw new UTFDataFormatException("encoded string too long: " + utflen + " bytes");
-        }
-
-        byte[] bytearr = null;
-        if (out instanceof DataOutputStream) {
-            DataOutputStream dos = (DataOutputStream) out;
-            if (dos.bytearr == null || (dos.bytearr.length < (utflen + 2))) {
-                dos.bytearr = new byte[(utflen * 2) + 2];
-            }
-            bytearr = dos.bytearr;
-        } else {
-            bytearr = new byte[utflen + 2];
-        }
-
-        bytearr[count++] = (byte) ((utflen >>> 8) & 0xFF);
-        bytearr[count++] = (byte) ((utflen) & 0xFF);
-
-        int i = 0;
-        for (i = 0; i < strlen; i++) {
-            c = str.charAt(i);
-            if (!((c >= 0x0001) && (c <= 0x007F))) {
-                break;
-            }
-            bytearr[count++] = (byte) c;
-        }
-
-        for (; i < strlen; i++) {
-            c = str.charAt(i);
-            if ((c >= 0x0001) && (c <= 0x007F)) {
-                bytearr[count++] = (byte) c;
-
-            } else if (c > 0x07FF) {
-                bytearr[count++] = (byte) (0xE0 | ((c >> 12) & 0x0F));
-                bytearr[count++] = (byte) (0x80 | ((c >> 6) & 0x3F));
-                bytearr[count++] = (byte) (0x80 | ((c) & 0x3F));
-            } else {
-                bytearr[count++] = (byte) (0xC0 | ((c >> 6) & 0x1F));
-                bytearr[count++] = (byte) (0x80 | ((c) & 0x3F));
-            }
-        }
-        out.write(bytearr, 0, utflen + 2);
-        return utflen + 2;
-    }
-
-    /**
-     * Returns the current value of the counter <code>written</code>, the number
-     * of bytes written to this data output stream so far. If the counter
-     * overflows, it will be wrapped to Integer.MAX_VALUE.
-     *
-     * @return the value of the <code>written</code> field.
-     * @see java.io.DataOutputStream#written
-     */
-    public final int size() {
-        return written;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/GZIPOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/GZIPOutputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/GZIPOutputStream.java
deleted file mode 100644
index 2864bbb..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/GZIPOutputStream.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.stream.io;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-/**
- * <p>
- * This class extends the {@link java.util.zip.GZIPOutputStream} by allowing the
- * constructor to provide a compression level, and uses a default value of 1,
- * rather than 5.
- * </p>
- */
-public class GZIPOutputStream extends java.util.zip.GZIPOutputStream {
-
-    public static final int DEFAULT_COMPRESSION_LEVEL = 1;
-
-    public GZIPOutputStream(final OutputStream out) throws IOException {
-        this(out, DEFAULT_COMPRESSION_LEVEL);
-    }
-
-    public GZIPOutputStream(final OutputStream out, final int compressionLevel) throws IOException {
-        super(out);
-        def.setLevel(compressionLevel);
-    }
-}


[44/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/BooleanLiteralEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/BooleanLiteralEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/BooleanLiteralEvaluator.java
deleted file mode 100644
index a085423..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/BooleanLiteralEvaluator.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.attribute.expression.language.evaluation.literals;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class BooleanLiteralEvaluator extends BooleanEvaluator {
-
-    private final boolean value;
-
-    public BooleanLiteralEvaluator(final boolean value) {
-        this.value = value;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        return new BooleanQueryResult(value);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/NumberLiteralEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/NumberLiteralEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/NumberLiteralEvaluator.java
deleted file mode 100644
index d7569e0..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/NumberLiteralEvaluator.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.attribute.expression.language.evaluation.literals;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class NumberLiteralEvaluator extends NumberEvaluator {
-
-    private final long literal;
-
-    public NumberLiteralEvaluator(final String value) {
-        this.literal = Long.parseLong(value);
-    }
-
-    @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        return new NumberQueryResult(literal);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/StringLiteralEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/StringLiteralEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/StringLiteralEvaluator.java
deleted file mode 100644
index d739ac7..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/StringLiteralEvaluator.java
+++ /dev/null
@@ -1,77 +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.attribute.expression.language.evaluation.literals;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class StringLiteralEvaluator extends StringEvaluator {
-
-    private final String value;
-
-    public StringLiteralEvaluator(final String value) {
-        // need to escape characters after backslashes
-        final StringBuilder sb = new StringBuilder();
-        boolean lastCharIsBackslash = false;
-        for (int i = 0; i < value.length(); i++) {
-            final char c = value.charAt(i);
-
-            if (lastCharIsBackslash) {
-                switch (c) {
-                    case 'n':
-                        sb.append("\n");
-                        break;
-                    case 'r':
-                        sb.append("\r");
-                        break;
-                    case '\\':
-                        sb.append("\\");
-                        break;
-                    case 't':
-                        sb.append("\\t");
-                        break;
-                    default:
-                        sb.append("\\").append(c);
-                        break;
-                }
-
-                lastCharIsBackslash = false;
-            } else if (c == '\\') {
-                lastCharIsBackslash = true;
-            } else {
-                sb.append(c);
-            }
-        }
-
-        this.value = sb.toString();
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        return new StringQueryResult(value);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/CountEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/CountEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/CountEvaluator.java
deleted file mode 100644
index f2af268..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/CountEvaluator.java
+++ /dev/null
@@ -1,56 +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.attribute.expression.language.evaluation.reduce;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-
-public class CountEvaluator extends NumberEvaluator implements ReduceEvaluator<Long> {
-
-    private final Evaluator<?> subjectEvaluator;
-    private long count = 0L;
-    
-    public CountEvaluator(final Evaluator<?> subjectEvaluator) {
-        this.subjectEvaluator = subjectEvaluator;
-    }
-    
-    @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final QueryResult<?> result = subjectEvaluator.evaluate(attributes);
-        if ( result.getValue() == null ) {
-            return new NumberQueryResult(count);
-        }
-        
-        if ( result.getResultType() == ResultType.BOOLEAN && ((Boolean) result.getValue()).equals(Boolean.FALSE) ) {
-            return new NumberQueryResult(count);
-        }
-
-        count++;
-        return new NumberQueryResult(count);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subjectEvaluator;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/JoinEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/JoinEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/JoinEvaluator.java
deleted file mode 100644
index eefdada..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/JoinEvaluator.java
+++ /dev/null
@@ -1,59 +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.attribute.expression.language.evaluation.reduce;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class JoinEvaluator extends StringEvaluator implements ReduceEvaluator<String> {
-    private final StringEvaluator subjectEvaluator;
-    private final StringEvaluator delimiterEvaluator;
-    
-    private final StringBuilder sb = new StringBuilder();
-    private int evalCount = 0;
-    
-    public JoinEvaluator(final StringEvaluator subject, final StringEvaluator delimiter) {
-        this.subjectEvaluator = subject;
-        this.delimiterEvaluator = delimiter;
-    }
-    
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        String subject = subjectEvaluator.evaluate(attributes).getValue();
-        if ( subject == null ) {
-            subject = "";
-        }
-        
-        final String delimiter = delimiterEvaluator.evaluate(attributes).getValue();
-        if ( evalCount > 0 ) {
-            sb.append(delimiter);
-        }
-        sb.append(subject);
-
-        evalCount++;
-        return new StringQueryResult( sb.toString() );
-    }
-    
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subjectEvaluator;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/ReduceEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/ReduceEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/ReduceEvaluator.java
deleted file mode 100644
index 12197c0..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/ReduceEvaluator.java
+++ /dev/null
@@ -1,23 +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.attribute.expression.language.evaluation.reduce;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-
-public interface ReduceEvaluator<T> extends Evaluator<T> {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java
deleted file mode 100644
index d9dd4d3..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java
+++ /dev/null
@@ -1,68 +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.attribute.expression.language.evaluation.selection;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class AllAttributesEvaluator extends BooleanEvaluator {
-
-    private final BooleanEvaluator booleanEvaluator;
-    private final MultiAttributeEvaluator multiAttributeEvaluator;
-
-    public AllAttributesEvaluator(final BooleanEvaluator booleanEvaluator, final MultiAttributeEvaluator multiAttributeEvaluator) {
-        this.booleanEvaluator = booleanEvaluator;
-        this.multiAttributeEvaluator = multiAttributeEvaluator;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        QueryResult<Boolean> attributeValueQuery = booleanEvaluator.evaluate(attributes);
-        Boolean result = attributeValueQuery.getValue();
-        if (result == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        if (!result) {
-            return new BooleanQueryResult(false);
-        }
-
-        while (multiAttributeEvaluator.getEvaluationsRemaining() > 0) {
-            attributeValueQuery = booleanEvaluator.evaluate(attributes);
-            result = attributeValueQuery.getValue();
-            if (result != null && !result) {
-                return attributeValueQuery;
-            }
-        }
-
-        return new BooleanQueryResult(true);
-    }
-
-    @Override
-    public int getEvaluationsRemaining() {
-        return 0;
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java
deleted file mode 100644
index 9192958..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java
+++ /dev/null
@@ -1,68 +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.attribute.expression.language.evaluation.selection;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class AnyAttributeEvaluator extends BooleanEvaluator {
-
-    private final BooleanEvaluator booleanEvaluator;
-    private final MultiAttributeEvaluator multiAttributeEvaluator;
-
-    public AnyAttributeEvaluator(final BooleanEvaluator booleanEvaluator, final MultiAttributeEvaluator multiAttributeEvaluator) {
-        this.booleanEvaluator = booleanEvaluator;
-        this.multiAttributeEvaluator = multiAttributeEvaluator;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        QueryResult<Boolean> attributeValueQuery = booleanEvaluator.evaluate(attributes);
-        Boolean result = attributeValueQuery.getValue();
-        if (result == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        if (result) {
-            return new BooleanQueryResult(true);
-        }
-
-        while (multiAttributeEvaluator.getEvaluationsRemaining() > 0) {
-            attributeValueQuery = booleanEvaluator.evaluate(attributes);
-            result = attributeValueQuery.getValue();
-            if (result != null && result) {
-                return attributeValueQuery;
-            }
-        }
-
-        return new BooleanQueryResult(false);
-    }
-
-    @Override
-    public int getEvaluationsRemaining() {
-        return 0;
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyMatchingAttributeEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyMatchingAttributeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyMatchingAttributeEvaluator.java
deleted file mode 100644
index 8c07278..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyMatchingAttributeEvaluator.java
+++ /dev/null
@@ -1,21 +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.attribute.expression.language.evaluation.selection;
-
-public class AnyMatchingAttributeEvaluator {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/DelineatedAttributeEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/DelineatedAttributeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/DelineatedAttributeEvaluator.java
deleted file mode 100644
index 209c86f..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/DelineatedAttributeEvaluator.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.attribute.expression.language.evaluation.selection;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class DelineatedAttributeEvaluator extends MultiAttributeEvaluator {
-
-    private final StringEvaluator subjectEvaluator;
-    private final StringEvaluator delimiterEvaluator;
-    private final int evaluationType;
-    private String[] delineatedValues;
-    private int evaluationCount = 0;
-    private int evaluationsLeft = 1;
-
-    public DelineatedAttributeEvaluator(final StringEvaluator subjectEvaluator, final StringEvaluator delimiterEvaluator, final int evaluationType) {
-        this.subjectEvaluator = subjectEvaluator;
-        this.delimiterEvaluator = delimiterEvaluator;
-        this.evaluationType = evaluationType;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        if (delineatedValues == null) {
-            final QueryResult<String> subjectValue = subjectEvaluator.evaluate(attributes);
-            if (subjectValue.getValue() == null) {
-                evaluationsLeft = 0;
-                return new StringQueryResult(null);
-            }
-
-            final QueryResult<String> delimiterValue = delimiterEvaluator.evaluate(attributes);
-            if (subjectValue.getValue() == null) {
-                evaluationsLeft = 0;
-                return new StringQueryResult(null);
-            }
-
-            delineatedValues = subjectValue.getValue().split(delimiterValue.getValue());
-        }
-
-        if (evaluationCount > delineatedValues.length) {
-            evaluationsLeft = 0;
-            return new StringQueryResult(null);
-        }
-
-        evaluationsLeft = delineatedValues.length - evaluationCount - 1;
-
-        return new StringQueryResult(delineatedValues[evaluationCount++]);
-    }
-
-    @Override
-    public int getEvaluationsRemaining() {
-        return evaluationsLeft;
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
-
-    @Override
-    public int getEvaluationType() {
-        return evaluationType;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java
deleted file mode 100644
index d872b6e..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java
+++ /dev/null
@@ -1,61 +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.attribute.expression.language.evaluation.selection;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.reduce.ReduceEvaluator;
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-
-public class MappingEvaluator<T> implements Evaluator<T> {
-    private final ReduceEvaluator<T> mappingEvaluator;
-    private final MultiAttributeEvaluator multiAttributeEvaluator;
-    
-    public MappingEvaluator(final ReduceEvaluator<T> mappingEvaluator, final MultiAttributeEvaluator multiAttributeEval) {
-        this.mappingEvaluator = mappingEvaluator;
-        this.multiAttributeEvaluator = multiAttributeEval;
-    }
-    
-    @Override
-    public QueryResult<T> evaluate(final Map<String, String> attributes) {
-        QueryResult<T> result = mappingEvaluator.evaluate(attributes);
-
-        while ( multiAttributeEvaluator.getEvaluationsRemaining() > 0 ) {
-            result = mappingEvaluator.evaluate(attributes);
-        }
-
-        return result;
-    }
-
-    @Override
-    public ResultType getResultType() {
-        return mappingEvaluator.getResultType();
-    }
-
-    @Override
-    public int getEvaluationsRemaining() {
-        return 0;
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiAttributeEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiAttributeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiAttributeEvaluator.java
deleted file mode 100644
index f80ed97..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiAttributeEvaluator.java
+++ /dev/null
@@ -1,24 +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.attribute.expression.language.evaluation.selection;
-
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-
-public abstract class MultiAttributeEvaluator extends StringEvaluator {
-
-    public abstract int getEvaluationType();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiMatchAttributeEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiMatchAttributeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiMatchAttributeEvaluator.java
deleted file mode 100644
index 9a441ce..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiMatchAttributeEvaluator.java
+++ /dev/null
@@ -1,82 +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.attribute.expression.language.evaluation.selection;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Pattern;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class MultiMatchAttributeEvaluator extends MultiAttributeEvaluator {
-
-    private final List<Pattern> attributePatterns;
-    private final int evaluationType;
-    private final List<String> attributeNames = new ArrayList<>();
-    private int evaluationCount = 0;
-
-    public MultiMatchAttributeEvaluator(final List<String> attributeRegexes, final int evaluationType) {
-        this.attributePatterns = new ArrayList<>();
-        for (final String regex : attributeRegexes) {
-            attributePatterns.add(Pattern.compile(regex));
-        }
-
-        this.evaluationType = evaluationType;
-    }
-
-    /**
-     * Can be called only after the first call to evaluate
-     *
-     * @return
-     */
-    @Override
-    public int getEvaluationsRemaining() {
-        return attributeNames.size() - evaluationCount;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        if (evaluationCount == 0) {
-            for (final Pattern pattern : attributePatterns) {
-                for (final String attrName : attributes.keySet()) {
-                    if (pattern.matcher(attrName).matches()) {
-                        attributeNames.add(attrName);
-                    }
-                }
-            }
-        }
-
-        if (evaluationCount >= attributeNames.size()) {
-            return new StringQueryResult(null);
-        }
-
-        return new StringQueryResult(attributes.get(attributeNames.get(evaluationCount++)));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
-
-    @Override
-    public int getEvaluationType() {
-        return evaluationType;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java
deleted file mode 100644
index 6dabc0a..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java
+++ /dev/null
@@ -1,64 +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.attribute.expression.language.evaluation.selection;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class MultiNamedAttributeEvaluator extends MultiAttributeEvaluator {
-
-    private final List<String> attributeNames;
-    private final int evaluationType;
-    private int evaluationCount = 0;
-    private List<String> matchingAttributeNames = null;
-
-    public MultiNamedAttributeEvaluator(final List<String> attributeNames, final int evaluationType) {
-        this.attributeNames = attributeNames;
-        this.evaluationType = evaluationType;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        matchingAttributeNames = new ArrayList<>(attributeNames);
-
-        if (matchingAttributeNames.size() <= evaluationCount) {
-            return new StringQueryResult(null);
-        }
-
-        return new StringQueryResult(attributes.get(matchingAttributeNames.get(evaluationCount++)));
-    }
-
-    @Override
-    public int getEvaluationsRemaining() {
-        return matchingAttributeNames.size() - evaluationCount;
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
-
-    @Override
-    public int getEvaluationType() {
-        return evaluationType;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/AttributeExpressionLanguageException.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/AttributeExpressionLanguageException.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/AttributeExpressionLanguageException.java
deleted file mode 100644
index 47d42cb..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/AttributeExpressionLanguageException.java
+++ /dev/null
@@ -1,34 +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.attribute.expression.language.exception;
-
-public class AttributeExpressionLanguageException extends RuntimeException {
-
-    private static final long serialVersionUID = -5637284498692447901L;
-
-    public AttributeExpressionLanguageException(final String explanation) {
-        super(explanation);
-    }
-
-    public AttributeExpressionLanguageException(final String explanation, final Throwable t) {
-        super(explanation, t);
-    }
-
-    public AttributeExpressionLanguageException(final Throwable t) {
-        super(t);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/AttributeExpressionLanguageParsingException.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/AttributeExpressionLanguageParsingException.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/AttributeExpressionLanguageParsingException.java
deleted file mode 100644
index f8531cb..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/AttributeExpressionLanguageParsingException.java
+++ /dev/null
@@ -1,34 +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.attribute.expression.language.exception;
-
-public class AttributeExpressionLanguageParsingException extends AttributeExpressionLanguageException {
-
-    private static final long serialVersionUID = 7422163230677064726L;
-
-    public AttributeExpressionLanguageParsingException(final String explanation) {
-        super(explanation);
-    }
-
-    public AttributeExpressionLanguageParsingException(final String explanation, final Throwable t) {
-        super(explanation, t);
-    }
-
-    public AttributeExpressionLanguageParsingException(final Throwable t) {
-        super(t);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/IllegalAttributeException.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/IllegalAttributeException.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/IllegalAttributeException.java
deleted file mode 100644
index f6f32ca..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/IllegalAttributeException.java
+++ /dev/null
@@ -1,29 +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.attribute.expression.language.exception;
-
-public class IllegalAttributeException extends RuntimeException {
-    private static final long serialVersionUID = 12348721897342L;
-
-    public IllegalAttributeException() {
-        super();
-    }
-
-    public IllegalAttributeException(final String explanation) {
-        super(explanation);
-    }
-}


[45/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ModEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ModEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ModEvaluator.java
deleted file mode 100644
index 6b7b838..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ModEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class ModEvaluator extends NumberEvaluator {
-
-    private final NumberEvaluator subject;
-    private final NumberEvaluator modValue;
-
-    public ModEvaluator(final NumberEvaluator subject, final NumberEvaluator modValue) {
-        this.subject = subject;
-        this.modValue = modValue;
-    }
-
-    @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final Long subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new NumberQueryResult(null);
-        }
-
-        final Long mod = modValue.evaluate(attributes).getValue();
-        if (mod == null) {
-            return new NumberQueryResult(null);
-        }
-
-        final long result = subjectValue % mod;
-        return new NumberQueryResult(result);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MultiplyEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MultiplyEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MultiplyEvaluator.java
deleted file mode 100644
index a3b03ed..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MultiplyEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class MultiplyEvaluator extends NumberEvaluator {
-
-    private final NumberEvaluator subject;
-    private final NumberEvaluator multiplyValue;
-
-    public MultiplyEvaluator(final NumberEvaluator subject, final NumberEvaluator multiplyValue) {
-        this.subject = subject;
-        this.multiplyValue = multiplyValue;
-    }
-
-    @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final Long subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new NumberQueryResult(null);
-        }
-
-        final Long multiply = multiplyValue.evaluate(attributes).getValue();
-        if (multiply == null) {
-            return new NumberQueryResult(null);
-        }
-
-        final long result = subjectValue * multiply;
-        return new NumberQueryResult(result);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotEvaluator.java
deleted file mode 100644
index 327e620..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class NotEvaluator extends BooleanEvaluator {
-
-    private final BooleanEvaluator subjectEvaluator;
-
-    public NotEvaluator(final BooleanEvaluator subjectEvaluator) {
-        this.subjectEvaluator = subjectEvaluator;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final QueryResult<Boolean> subjectValue = subjectEvaluator.evaluate(attributes);
-        if (subjectValue == null) {
-            return new BooleanQueryResult(null);
-        }
-
-        return new BooleanQueryResult(!subjectValue.getValue());
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subjectEvaluator;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotNullEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotNullEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotNullEvaluator.java
deleted file mode 100644
index 126fbfa..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotNullEvaluator.java
+++ /dev/null
@@ -1,45 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class NotNullEvaluator extends BooleanEvaluator {
-
-    private final Evaluator<?> subject;
-
-    public NotNullEvaluator(final Evaluator<?> subject) {
-        this.subject = subject;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final Object subjectValue = subject.evaluate(attributes).getValue();
-        return new BooleanQueryResult(subjectValue != null);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NowEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NowEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NowEvaluator.java
deleted file mode 100644
index bfde6b0..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NowEvaluator.java
+++ /dev/null
@@ -1,39 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Date;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.DateQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class NowEvaluator extends DateEvaluator {
-
-    @Override
-    public QueryResult<Date> evaluate(final Map<String, String> attributes) {
-        return new DateQueryResult(new Date());
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NumberToDateEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NumberToDateEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NumberToDateEvaluator.java
deleted file mode 100644
index 672d855..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NumberToDateEvaluator.java
+++ /dev/null
@@ -1,52 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Date;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.DateQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class NumberToDateEvaluator extends DateEvaluator {
-
-    private final NumberEvaluator subject;
-
-    public NumberToDateEvaluator(final NumberEvaluator subject) {
-        this.subject = subject;
-    }
-
-    @Override
-    public QueryResult<Date> evaluate(final Map<String, String> attributes) {
-        final QueryResult<Long> result = subject.evaluate(attributes);
-        final Long value = result.getValue();
-        if (value == null) {
-            return null;
-        }
-
-        return new DateQueryResult(new Date(value));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OneUpSequenceEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OneUpSequenceEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OneUpSequenceEvaluator.java
deleted file mode 100644
index 11d3fa9..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OneUpSequenceEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class OneUpSequenceEvaluator extends NumberEvaluator {
-
-    private static final AtomicLong value = new AtomicLong(0L);
-
-    @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        return new NumberQueryResult(value.getAndIncrement());
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OrEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OrEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OrEvaluator.java
deleted file mode 100644
index ab3a16c..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OrEvaluator.java
+++ /dev/null
@@ -1,60 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class OrEvaluator extends BooleanEvaluator {
-
-    private final BooleanEvaluator subjectEvaluator;
-    private final BooleanEvaluator rhsEvaluator;
-
-    public OrEvaluator(final BooleanEvaluator subjectEvaluator, final BooleanEvaluator rhsEvaluator) {
-        this.subjectEvaluator = subjectEvaluator;
-        this.rhsEvaluator = rhsEvaluator;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final QueryResult<Boolean> subjectValue = subjectEvaluator.evaluate(attributes);
-        if (subjectValue == null) {
-            return new BooleanQueryResult(null);
-        }
-
-        if (Boolean.TRUE.equals(subjectValue.getValue())) {
-            return new BooleanQueryResult(true);
-        }
-
-        final QueryResult<Boolean> rhsValue = rhsEvaluator.evaluate(attributes);
-        if (rhsValue == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        return new BooleanQueryResult(rhsValue.getValue());
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subjectEvaluator;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java
deleted file mode 100644
index dff693d..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class PlusEvaluator extends NumberEvaluator {
-
-    private final NumberEvaluator subject;
-    private final NumberEvaluator plusValue;
-
-    public PlusEvaluator(final NumberEvaluator subject, final NumberEvaluator plusValue) {
-        this.subject = subject;
-        this.plusValue = plusValue;
-    }
-
-    @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final Long subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new NumberQueryResult(null);
-        }
-
-        final Long plus = plusValue.evaluate(attributes).getValue();
-        if (plus == null) {
-            return new NumberQueryResult(null);
-        }
-
-        final long result = subjectValue + plus;
-        return new NumberQueryResult(result);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java
deleted file mode 100644
index 5b66b8f..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java
+++ /dev/null
@@ -1,50 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class PrependEvaluator extends StringEvaluator {
-
-    private final StringEvaluator subject;
-    private final StringEvaluator prependEvaluator;
-
-    public PrependEvaluator(final StringEvaluator subject, final StringEvaluator prepend) {
-        this.subject = subject;
-        this.prependEvaluator = prepend;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        final String prependValue = prependEvaluator.evaluate(attributes).getValue();
-
-        final String result = (prependValue == null ? "" : prependValue) + (subjectValue == null ? "" : subjectValue);
-        return new StringQueryResult(result);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java
deleted file mode 100644
index 19ed63e..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java
+++ /dev/null
@@ -1,55 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class ReplaceAllEvaluator extends StringEvaluator {
-
-    private final StringEvaluator subject;
-    private final StringEvaluator search;
-    private final StringEvaluator replacement;
-
-    public ReplaceAllEvaluator(final StringEvaluator subject, final StringEvaluator search, final StringEvaluator replacement) {
-        this.subject = subject;
-        this.search = search;
-        this.replacement = replacement;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new StringQueryResult(null);
-        }
-        final String searchValue = search.evaluate(attributes).getValue();
-        final String replacementValue = replacement.evaluate(attributes).getValue();
-
-        return new StringQueryResult(subjectValue.replaceAll(searchValue, replacementValue));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEmptyEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEmptyEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEmptyEvaluator.java
deleted file mode 100644
index e5c40d2..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEmptyEvaluator.java
+++ /dev/null
@@ -1,50 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-
-public class ReplaceEmptyEvaluator extends StringEvaluator {
-    private final StringEvaluator subjectEvaluator;
-    private final StringEvaluator replacementEvaluator;
-    
-    public ReplaceEmptyEvaluator(final StringEvaluator subjectEvaluator, final StringEvaluator replacementEvaluator) {
-        this.subjectEvaluator = subjectEvaluator;
-        this.replacementEvaluator = replacementEvaluator;
-    }
-    
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final QueryResult<String> subjectResult = subjectEvaluator.evaluate(attributes);
-        final String subjectValue = subjectResult.getValue();
-        final boolean isEmpty = subjectValue == null || subjectValue.toString().trim().isEmpty();
-        if ( isEmpty ) {
-            return replacementEvaluator.evaluate(attributes);
-        } else {
-            return subjectResult;
-        }
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subjectEvaluator;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEvaluator.java
deleted file mode 100644
index d6a7c5b..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEvaluator.java
+++ /dev/null
@@ -1,55 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class ReplaceEvaluator extends StringEvaluator {
-
-    private final StringEvaluator subject;
-    private final StringEvaluator search;
-    private final StringEvaluator replacement;
-
-    public ReplaceEvaluator(final StringEvaluator subject, final StringEvaluator search, final StringEvaluator replacement) {
-        this.subject = subject;
-        this.search = search;
-        this.replacement = replacement;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new StringQueryResult(null);
-        }
-        final String searchValue = search.evaluate(attributes).getValue();
-        final String replacementValue = replacement.evaluate(attributes).getValue();
-
-        return new StringQueryResult(subjectValue.replace(searchValue, replacementValue));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceNullEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceNullEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceNullEvaluator.java
deleted file mode 100644
index 91333b7..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceNullEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class ReplaceNullEvaluator extends StringEvaluator {
-
-    private final StringEvaluator subject;
-    private final StringEvaluator resultEvaluator;
-
-    public ReplaceNullEvaluator(final StringEvaluator subject, final StringEvaluator resultEvaluator) {
-        this.subject = subject;
-        this.resultEvaluator = resultEvaluator;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        return new StringQueryResult(subjectValue == null ? resultEvaluator.evaluate(attributes).getValue() : subjectValue);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StartsWithEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StartsWithEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StartsWithEvaluator.java
deleted file mode 100644
index c6a7eb7..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StartsWithEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-
-public class StartsWithEvaluator extends BooleanEvaluator {
-
-    private final StringEvaluator subject;
-    private final StringEvaluator search;
-
-    public StartsWithEvaluator(final StringEvaluator subject, final StringEvaluator search) {
-        this.subject = subject;
-        this.search = search;
-    }
-
-    @Override
-    public QueryResult<Boolean> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new BooleanQueryResult(false);
-        }
-
-        final String searchString = search.evaluate(attributes).getValue();
-        return new BooleanQueryResult(searchString == null ? false : subjectValue.startsWith(searchString));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StringToDateEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StringToDateEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StringToDateEvaluator.java
deleted file mode 100644
index 1575d75..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StringToDateEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Locale;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.DateQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.exception.IllegalAttributeException;
-
-public class StringToDateEvaluator extends DateEvaluator {
-
-    private final StringEvaluator subject;
-    private final StringEvaluator format;
-
-    public StringToDateEvaluator(final StringEvaluator subject, final StringEvaluator format) {
-        this.subject = subject;
-        this.format = format;
-    }
-
-    @Override
-    public QueryResult<Date> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        final String formatValue = format.evaluate(attributes).getValue();
-        if (subjectValue == null || formatValue == null) {
-            return new DateQueryResult(null);
-        }
-
-        try {
-            return new DateQueryResult(new SimpleDateFormat(formatValue, Locale.US).parse(subjectValue));
-        } catch (final ParseException e) {
-            throw new IllegalAttributeException("Cannot parse attribute value as a date; date format: "
-                    + formatValue + "; attribute value: " + subjectValue);
-        } catch (final IllegalArgumentException e) {
-            throw new IllegalAttributeException("Invalid date format: " + formatValue);
-        }
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterEvaluator.java
deleted file mode 100644
index a8b7357..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterEvaluator.java
+++ /dev/null
@@ -1,59 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class SubstringAfterEvaluator extends StringEvaluator {
-
-    private final StringEvaluator subject;
-    private final StringEvaluator afterEvaluator;
-
-    public SubstringAfterEvaluator(final StringEvaluator subject, final StringEvaluator afterEvaluator) {
-        this.subject = subject;
-        this.afterEvaluator = afterEvaluator;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new StringQueryResult("");
-        }
-        final String afterValue = afterEvaluator.evaluate(attributes).getValue();
-        if (afterValue == null || afterValue.length() == 0) {
-            return new StringQueryResult(subjectValue);
-        }
-
-        final int index = subjectValue.indexOf(afterValue);
-        if (index < 0) {
-            return new StringQueryResult(subjectValue);
-        }
-        return new StringQueryResult(subjectValue.substring(index + afterValue.length()));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterLastEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterLastEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterLastEvaluator.java
deleted file mode 100644
index 3dac40e..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterLastEvaluator.java
+++ /dev/null
@@ -1,55 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class SubstringAfterLastEvaluator extends StringEvaluator {
-
-    private final StringEvaluator subject;
-    private final StringEvaluator afterEvaluator;
-
-    public SubstringAfterLastEvaluator(final StringEvaluator subject, final StringEvaluator afterEvaluator) {
-        this.subject = subject;
-        this.afterEvaluator = afterEvaluator;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new StringQueryResult("");
-        }
-        final String afterValue = afterEvaluator.evaluate(attributes).getValue();
-        final int index = subjectValue.lastIndexOf(afterValue);
-        if (index < 0 || index >= subjectValue.length()) {
-            return new StringQueryResult(subjectValue);
-        }
-        return new StringQueryResult(subjectValue.substring(index + afterValue.length()));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeEvaluator.java
deleted file mode 100644
index 92b136c..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class SubstringBeforeEvaluator extends StringEvaluator {
-
-    private final StringEvaluator subject;
-    private final StringEvaluator beforeEvaluator;
-
-    public SubstringBeforeEvaluator(final StringEvaluator subject, final StringEvaluator beforeEvaluator) {
-        this.subject = subject;
-        this.beforeEvaluator = beforeEvaluator;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new StringQueryResult("");
-        }
-        final String beforeValue = beforeEvaluator.evaluate(attributes).getValue();
-        if (beforeValue == null || beforeValue.length() == 0) {
-            return new StringQueryResult(subjectValue);
-        }
-        final int index = subjectValue.indexOf(beforeValue);
-        if (index < 0) {
-            return new StringQueryResult(subjectValue);
-        }
-        return new StringQueryResult(subjectValue.substring(0, index));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeLastEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeLastEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeLastEvaluator.java
deleted file mode 100644
index 818a3dc..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeLastEvaluator.java
+++ /dev/null
@@ -1,55 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class SubstringBeforeLastEvaluator extends StringEvaluator {
-
-    private final StringEvaluator subject;
-    private final StringEvaluator beforeEvaluator;
-
-    public SubstringBeforeLastEvaluator(final StringEvaluator subject, final StringEvaluator beforeEvaluator) {
-        this.subject = subject;
-        this.beforeEvaluator = beforeEvaluator;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new StringQueryResult("");
-        }
-        final String beforeValue = beforeEvaluator.evaluate(attributes).getValue();
-        final int index = subjectValue.lastIndexOf(beforeValue);
-        if (index < 0) {
-            return new StringQueryResult(subjectValue);
-        }
-        return new StringQueryResult(subjectValue.substring(0, index));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringEvaluator.java
deleted file mode 100644
index da11c45..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class SubstringEvaluator extends StringEvaluator {
-
-    private final StringEvaluator subject;
-    private final NumberEvaluator startIndex;
-    private final NumberEvaluator endIndex;
-
-    public SubstringEvaluator(final StringEvaluator subject, final NumberEvaluator startIndex, final NumberEvaluator endIndex) {
-        this.subject = subject;
-        this.startIndex = startIndex;
-        this.endIndex = endIndex;
-    }
-
-    public SubstringEvaluator(final StringEvaluator subject, final NumberEvaluator startIndex) {
-        this.subject = subject;
-        this.startIndex = startIndex;
-        this.endIndex = null;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new StringQueryResult("");
-        }
-        final int startIndexValue = startIndex.evaluate(attributes).getValue().intValue();
-        if (endIndex == null) {
-            return new StringQueryResult(subjectValue.substring(startIndexValue));
-        } else {
-            final int endIndexValue = endIndex.evaluate(attributes).getValue().intValue();
-            return new StringQueryResult(subjectValue.substring(startIndexValue, endIndexValue));
-        }
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToLowerEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToLowerEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToLowerEvaluator.java
deleted file mode 100644
index 7de01eb..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToLowerEvaluator.java
+++ /dev/null
@@ -1,45 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class ToLowerEvaluator extends StringEvaluator {
-
-    private final StringEvaluator subject;
-
-    public ToLowerEvaluator(final StringEvaluator subject) {
-        this.subject = subject;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        return new StringQueryResult((subjectValue == null) ? null : subjectValue.toLowerCase());
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToNumberEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToNumberEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToNumberEvaluator.java
deleted file mode 100644
index 1d4664a..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToNumberEvaluator.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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-
-public class ToNumberEvaluator extends NumberEvaluator {
-
-    private final StringEvaluator subject;
-
-    public ToNumberEvaluator(final StringEvaluator subject) {
-        this.subject = subject;
-    }
-
-    @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        return new NumberQueryResult((subjectValue == null || subjectValue.trim().isEmpty()) ? null : Long.valueOf(subjectValue));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToRadixEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToRadixEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToRadixEvaluator.java
deleted file mode 100644
index aab094d..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToRadixEvaluator.java
+++ /dev/null
@@ -1,77 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Arrays;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class ToRadixEvaluator extends StringEvaluator {
-
-    private final NumberEvaluator numberEvaluator;
-    private final NumberEvaluator radixEvaluator;
-    private final NumberEvaluator minimumWidthEvaluator;
-
-    public ToRadixEvaluator(final NumberEvaluator subject, final NumberEvaluator radixEvaluator) {
-        this(subject, radixEvaluator, null);
-    }
-
-    public ToRadixEvaluator(final NumberEvaluator subject, final NumberEvaluator radixEvaluator, final NumberEvaluator minimumWidthEvaluator) {
-        this.numberEvaluator = subject;
-        this.radixEvaluator = radixEvaluator;
-        this.minimumWidthEvaluator = minimumWidthEvaluator;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final Long result = numberEvaluator.evaluate(attributes).getValue();
-        if (result == null) {
-            return new StringQueryResult(null);
-        }
-
-        final Long radix = radixEvaluator.evaluate(attributes).getValue();
-        if (radix == null) {
-            return new StringQueryResult(null);
-        }
-
-        String stringValue = Long.toString(result.longValue(), radix.intValue());
-        if (minimumWidthEvaluator != null) {
-            final Long minimumWidth = minimumWidthEvaluator.evaluate(attributes).getValue();
-            if (minimumWidth != null) {
-                final int paddingWidth = minimumWidth.intValue() - stringValue.length();
-                if (paddingWidth > 0) {
-                    final char[] padChars = new char[paddingWidth];
-                    Arrays.fill(padChars, '0');
-                    stringValue = String.valueOf(padChars) + stringValue;
-                }
-            }
-        }
-
-        return new StringQueryResult(stringValue);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return numberEvaluator;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToStringEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToStringEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToStringEvaluator.java
deleted file mode 100644
index 2f7fbe3..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToStringEvaluator.java
+++ /dev/null
@@ -1,45 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class ToStringEvaluator extends StringEvaluator {
-
-    private final Evaluator<?> subject;
-
-    public ToStringEvaluator(final Evaluator<?> subject) {
-        this.subject = subject;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final Object result = subject.evaluate(attributes).getValue();
-        return new StringQueryResult(result == null ? null : result.toString());
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToUpperEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToUpperEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToUpperEvaluator.java
deleted file mode 100644
index 29bc61d..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToUpperEvaluator.java
+++ /dev/null
@@ -1,45 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class ToUpperEvaluator extends StringEvaluator {
-
-    private final StringEvaluator subject;
-
-    public ToUpperEvaluator(final StringEvaluator subject) {
-        this.subject = subject;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        return new StringQueryResult((subjectValue == null) ? null : subjectValue.toUpperCase());
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/TrimEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/TrimEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/TrimEvaluator.java
deleted file mode 100644
index b9926d7..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/TrimEvaluator.java
+++ /dev/null
@@ -1,45 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class TrimEvaluator extends StringEvaluator {
-
-    private final StringEvaluator subject;
-
-    public TrimEvaluator(final StringEvaluator subject) {
-        this.subject = subject;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        return new StringQueryResult(subjectValue == null ? null : subjectValue.trim());
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlDecodeEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlDecodeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlDecodeEvaluator.java
deleted file mode 100644
index 6c3ba14..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlDecodeEvaluator.java
+++ /dev/null
@@ -1,55 +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.attribute.expression.language.evaluation.functions;
-
-import java.io.UnsupportedEncodingException;
-import java.net.URLDecoder;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class UrlDecodeEvaluator extends StringEvaluator {
-
-    private final StringEvaluator subject;
-
-    public UrlDecodeEvaluator(final StringEvaluator subject) {
-        this.subject = subject;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new StringQueryResult(null);
-        }
-
-        try {
-            return new StringQueryResult(URLDecoder.decode(subjectValue, "UTF-8"));
-        } catch (UnsupportedEncodingException e) {
-            return null;    // won't happen. It's UTF-8
-        }
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlEncodeEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlEncodeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlEncodeEvaluator.java
deleted file mode 100644
index d1e2caa..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlEncodeEvaluator.java
+++ /dev/null
@@ -1,55 +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.attribute.expression.language.evaluation.functions;
-
-import java.io.UnsupportedEncodingException;
-import java.net.URLEncoder;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class UrlEncodeEvaluator extends StringEvaluator {
-
-    private final StringEvaluator subject;
-
-    public UrlEncodeEvaluator(final StringEvaluator subject) {
-        this.subject = subject;
-    }
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        if (subjectValue == null) {
-            return new StringQueryResult(null);
-        }
-
-        try {
-            return new StringQueryResult(URLEncoder.encode(subjectValue, "UTF-8"));
-        } catch (UnsupportedEncodingException e) {
-            return null;    // won't happen. It's UTF-8
-        }
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UuidEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UuidEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UuidEvaluator.java
deleted file mode 100644
index faa8f0a..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UuidEvaluator.java
+++ /dev/null
@@ -1,39 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-import java.util.UUID;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class UuidEvaluator extends StringEvaluator {
-
-    @Override
-    public QueryResult<String> evaluate(final Map<String, String> attributes) {
-        return new StringQueryResult(UUID.randomUUID().toString());
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
-
-}


[26/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/InvalidateUserAccountActionTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/InvalidateUserAccountActionTest.java b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/InvalidateUserAccountActionTest.java
deleted file mode 100644
index 93dbe61..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/InvalidateUserAccountActionTest.java
+++ /dev/null
@@ -1,131 +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.admin.service.action;
-
-import java.util.Date;
-import org.junit.Assert;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.user.NiFiUser;
-import org.apache.commons.lang3.StringUtils;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Test case for InvalidateUserAccountAction.
- */
-public class InvalidateUserAccountActionTest {
-
-    private static final String USER_ID_1 = "1";
-    private static final String USER_ID_2 = "2";
-    private static final String USER_ID_3 = "3";
-
-    private DAOFactory daoFactory;
-    private UserDAO userDao;
-
-    @Before
-    public void setup() throws Exception {
-        // mock the user dao
-        userDao = Mockito.mock(UserDAO.class);
-        Mockito.doAnswer(new Answer<NiFiUser>() {
-            @Override
-            public NiFiUser answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String id = (String) args[0];
-
-                NiFiUser user = null;
-                if (USER_ID_1.equals(id)) {
-                    // leave uninitialized
-                } else if (USER_ID_2.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_2);
-                } else if (USER_ID_3.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_3);
-                    user.setLastVerified(new Date());
-                }
-                return user;
-            }
-        }).when(userDao).findUserById(Mockito.anyString());
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                NiFiUser user = (NiFiUser) args[0];
-
-                if (USER_ID_2.equals(user.getId())) {
-                    throw new DataAccessException(StringUtils.EMPTY);
-                }
-
-                // do nothing
-                return null;
-            }
-        }).when(userDao).updateUser(Mockito.any(NiFiUser.class));
-
-        // mock the dao factory
-        daoFactory = Mockito.mock(DAOFactory.class);
-        Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao);
-    }
-
-    /**
-     * Tests when the user account cannot be found.
-     *
-     * @throws Exception
-     */
-    @Test(expected = AccountNotFoundException.class)
-    public void testAccountNotFoundException() throws Exception {
-        InvalidateUserAccountAction invalidateUserAccount = new InvalidateUserAccountAction(USER_ID_1);
-        invalidateUserAccount.execute(daoFactory, null);
-    }
-
-    /**
-     * Tests when a data access exception occurs when updating the user record.
-     *
-     * @throws Exception
-     */
-    @Test(expected = DataAccessException.class)
-    public void testDataAccessException() throws Exception {
-        InvalidateUserAccountAction invalidateUserAccount = new InvalidateUserAccountAction(USER_ID_2);
-        invalidateUserAccount.execute(daoFactory, null);
-    }
-
-    /**
-     * Tests the general case of invalidating a user.
-     *
-     * @throws Exception
-     */
-    @Test
-    public void testInvalidateUser() throws Exception {
-        InvalidateUserAccountAction invalidateUserAccount = new InvalidateUserAccountAction(USER_ID_3);
-        invalidateUserAccount.execute(daoFactory, null);
-
-        // verify the interaction with the dao
-        ArgumentCaptor<NiFiUser> userCaptor = ArgumentCaptor.forClass(NiFiUser.class);
-        Mockito.verify(userDao, Mockito.times(1)).updateUser(userCaptor.capture());
-
-        // verify the user
-        NiFiUser user = userCaptor.getValue();
-        Assert.assertEquals(USER_ID_3, user.getId());
-        Assert.assertNull(user.getLastVerified());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/RequestUserAccountActionTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/RequestUserAccountActionTest.java b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/RequestUserAccountActionTest.java
deleted file mode 100644
index 6e77d46..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/RequestUserAccountActionTest.java
+++ /dev/null
@@ -1,127 +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.admin.service.action;
-
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUser;
-import org.apache.commons.lang3.StringUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Test case for RequestUserAccountAction.
- */
-public class RequestUserAccountActionTest {
-
-    private static final String USER_ID_3 = "3";
-
-    private static final String USER_DN_1 = "existing user account dn";
-    private static final String USER_DN_2 = "data access exception";
-    private static final String USER_DN_3 = "new account request";
-
-    private DAOFactory daoFactory;
-    private UserDAO userDao;
-
-    @Before
-    public void setup() throws Exception {
-        // mock the user dao
-        userDao = Mockito.mock(UserDAO.class);
-        Mockito.doAnswer(new Answer<NiFiUser>() {
-            @Override
-            public NiFiUser answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String dn = (String) args[0];
-
-                NiFiUser user = null;
-                if (USER_DN_1.equals(dn)) {
-                    user = new NiFiUser();
-                }
-                return user;
-            }
-        }).when(userDao).findUserByDn(Mockito.anyString());
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                NiFiUser user = (NiFiUser) args[0];
-                switch (user.getDn()) {
-                    case USER_DN_2:
-                        throw new DataAccessException();
-                    case USER_DN_3:
-                        user.setId(USER_ID_3);
-                        break;
-                }
-
-                // do nothing
-                return null;
-            }
-        }).when(userDao).createUser(Mockito.any(NiFiUser.class));
-
-        // mock the dao factory
-        daoFactory = Mockito.mock(DAOFactory.class);
-        Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao);
-    }
-
-    /**
-     * Tests when a user account already exists.
-     *
-     * @throws Exception
-     */
-    @Test(expected = IllegalArgumentException.class)
-    public void testExistingAccount() throws Exception {
-        RequestUserAccountAction requestUserAccount = new RequestUserAccountAction(USER_DN_1, StringUtils.EMPTY);
-        requestUserAccount.execute(daoFactory, null);
-    }
-
-    /**
-     * Tests when a DataAccessException occurs while saving the new account
-     * request.
-     *
-     * @throws Exception
-     */
-    @Test(expected = DataAccessException.class)
-    public void testDataAccessException() throws Exception {
-        RequestUserAccountAction requestUserAccount = new RequestUserAccountAction(USER_DN_2, StringUtils.EMPTY);
-        requestUserAccount.execute(daoFactory, null);
-    }
-
-    /**
-     * Tests the general case for requesting a new user account.
-     *
-     * @throws Exception
-     */
-    @Test
-    public void testRequestUserAccountAction() throws Exception {
-        RequestUserAccountAction requestUserAccount = new RequestUserAccountAction(USER_DN_3, StringUtils.EMPTY);
-        NiFiUser user = requestUserAccount.execute(daoFactory, null);
-
-        // verfiy the user
-        Assert.assertEquals(USER_ID_3, user.getId());
-        Assert.assertEquals(USER_DN_3, user.getDn());
-        Assert.assertEquals(AccountStatus.PENDING, user.getStatus());
-
-        // verify interaction with dao
-        Mockito.verify(userDao, Mockito.times(1)).createUser(user);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/SeedUserAccountsActionTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/SeedUserAccountsActionTest.java b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/SeedUserAccountsActionTest.java
deleted file mode 100644
index f37fc84..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/SeedUserAccountsActionTest.java
+++ /dev/null
@@ -1,263 +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.admin.service.action;
-
-import org.apache.nifi.admin.service.action.SeedUserAccountsAction;
-import java.util.EnumSet;
-import java.util.HashSet;
-import java.util.Set;
-import org.apache.nifi.admin.dao.AuthorityDAO;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.authorization.Authority;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUser;
-import org.hamcrest.Matcher;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.ArgumentMatcher;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- *
- */
-public class SeedUserAccountsActionTest {
-
-    private static final String USER_ID_1 = "1";
-    private static final String USER_ID_2 = "2";
-    private static final String USER_ID_3 = "3";
-    private static final String USER_ID_4 = "4";
-
-    private static final String USER_DN_1 = "user dn 1 - active user - remove monitor and operator, add dfm";
-    private static final String USER_DN_2 = "user dn 2 - active user - no action";
-    private static final String USER_DN_3 = "user dn 3 - pending user - add operator";
-    private static final String USER_DN_4 = "user dn 4 - new user - add monitor";
-
-    private DAOFactory daoFactory;
-    private UserDAO userDao;
-    private AuthorityDAO authorityDao;
-    private AuthorityProvider authorityProvider;
-
-    @Before
-    public void setup() throws Exception {
-        // mock the user dao
-        userDao = Mockito.mock(UserDAO.class);
-        Mockito.doAnswer(new Answer<NiFiUser>() {
-            @Override
-            public NiFiUser answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String id = (String) args[0];
-
-                NiFiUser user = null;
-                if (USER_ID_1.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_1);
-                    user.setDn(USER_DN_1);
-                    user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
-                    user.setStatus(AccountStatus.ACTIVE);
-                } else if (USER_ID_2.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_2);
-                    user.setDn(USER_DN_2);
-                    user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_ADMIN));
-                    user.setStatus(AccountStatus.ACTIVE);
-                } else if (USER_ID_3.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_3);
-                    user.setDn(USER_DN_3);
-                    user.setStatus(AccountStatus.PENDING);
-                }
-                return user;
-            }
-        }).when(userDao).findUserById(Mockito.anyString());
-        Mockito.doAnswer(new Answer<NiFiUser>() {
-            @Override
-            public NiFiUser answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String dn = (String) args[0];
-
-                NiFiUser user = null;
-                if (USER_DN_1.equals(dn)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_1);
-                    user.setDn(USER_DN_1);
-                    user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
-                    user.setStatus(AccountStatus.ACTIVE);
-                } else if (USER_DN_2.equals(dn)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_2);
-                    user.setDn(USER_DN_2);
-                    user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_ADMIN));
-                    user.setStatus(AccountStatus.ACTIVE);
-                } else if (USER_DN_3.equals(dn)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_3);
-                    user.setDn(USER_DN_3);
-                    user.setStatus(AccountStatus.PENDING);
-                }
-                return user;
-            }
-        }).when(userDao).findUserByDn(Mockito.anyString());
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                NiFiUser user = (NiFiUser) args[0];
-
-                if (USER_DN_4.equals(user.getDn())) {
-                    user.setId(USER_ID_4);
-                }
-
-                return null;
-            }
-        }).when(userDao).createUser(Mockito.any(NiFiUser.class));
-
-        // mock the authority dao
-        authorityDao = Mockito.mock(AuthorityDAO.class);
-
-        // mock the authority provider
-        authorityProvider = Mockito.mock(AuthorityProvider.class);
-        Mockito.doAnswer(new Answer<Set<String>>() {
-            @Override
-            public Set<String> answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                Authority role = (Authority) args[0];
-
-                Set<String> users = new HashSet<>();
-                if (Authority.ROLE_DFM.equals(role)) {
-                    users.add(USER_DN_1);
-                } else if (Authority.ROLE_ADMIN.equals(role)) {
-                    users.add(USER_DN_2);
-                } else if (Authority.ROLE_PROXY.equals(role)) {
-                    users.add(USER_DN_3);
-                } else if (Authority.ROLE_MONITOR.equals(role)) {
-                    users.add(USER_DN_4);
-                }
-                return users;
-            }
-        }).when(authorityProvider).getUsers(Mockito.any(Authority.class));
-        Mockito.doAnswer(new Answer<Set<Authority>>() {
-            @Override
-            public Set<Authority> answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String dn = (String) args[0];
-
-                Set<Authority> authorities = EnumSet.noneOf(Authority.class);
-                switch (dn) {
-                    case USER_DN_1:
-                        authorities.add(Authority.ROLE_DFM);
-                        break;
-                    case USER_DN_2:
-                        authorities.add(Authority.ROLE_ADMIN);
-                        break;
-                    case USER_DN_3:
-                        authorities.add(Authority.ROLE_PROXY);
-                        break;
-                    case USER_DN_4:
-                        authorities.add(Authority.ROLE_MONITOR);
-                        break;
-                }
-                return authorities;
-            }
-        }).when(authorityProvider).getAuthorities(Mockito.anyString());
-
-        // mock the dao factory
-        daoFactory = Mockito.mock(DAOFactory.class);
-        Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao);
-        Mockito.when(daoFactory.getAuthorityDAO()).thenReturn(authorityDao);
-    }
-
-    /**
-     * Tests seeding the user accounts.
-     *
-     * @throws Exception
-     */
-    @Test
-    public void testSeedUsers() throws Exception {
-        SeedUserAccountsAction seedUserAccounts = new SeedUserAccountsAction();
-        seedUserAccounts.execute(daoFactory, authorityProvider);
-
-        // matcher for user 1
-        Matcher<NiFiUser> matchesUser1 = new ArgumentMatcher<NiFiUser>() {
-            @Override
-            public boolean matches(Object argument) {
-                NiFiUser user = (NiFiUser) argument;
-                return USER_ID_1.equals(user.getId());
-            }
-        };
-
-        // verify user 1 - active existing user - remove monitor, operator, add dfm
-        Mockito.verify(userDao, Mockito.times(1)).updateUser(Mockito.argThat(matchesUser1));
-        Mockito.verify(userDao, Mockito.never()).createUser(Mockito.argThat(matchesUser1));
-        Mockito.verify(authorityDao, Mockito.times(1)).createAuthorities(EnumSet.of(Authority.ROLE_DFM), USER_ID_1);
-
-        // matcher for user 2
-        Matcher<NiFiUser> matchesUser2 = new ArgumentMatcher<NiFiUser>() {
-            @Override
-            public boolean matches(Object argument) {
-                NiFiUser user = (NiFiUser) argument;
-                return USER_ID_2.equals(user.getId());
-            }
-        };
-
-        // verify user 2 - active existing user - no actions
-        Mockito.verify(userDao, Mockito.times(1)).updateUser(Mockito.argThat(matchesUser2));
-        Mockito.verify(userDao, Mockito.never()).createUser(Mockito.argThat(matchesUser2));
-        Mockito.verify(authorityDao, Mockito.never()).createAuthorities(Mockito.anySet(), Mockito.eq(USER_ID_2));
-        Mockito.verify(authorityDao, Mockito.never()).deleteAuthorities(Mockito.anySet(), Mockito.eq(USER_ID_2));
-
-        // matchers for user 3
-        Matcher<NiFiUser> matchesPendingUser3 = new ArgumentMatcher<NiFiUser>() {
-            @Override
-            public boolean matches(Object argument) {
-                NiFiUser user = (NiFiUser) argument;
-                return USER_ID_3.equals(user.getId()) && AccountStatus.ACTIVE.equals(user.getStatus());
-            }
-        };
-        Matcher<NiFiUser> matchesUser3 = new ArgumentMatcher<NiFiUser>() {
-            @Override
-            public boolean matches(Object argument) {
-                NiFiUser user = (NiFiUser) argument;
-                return USER_ID_3.equals(user.getId());
-            }
-        };
-
-        // verify user 3 - pending user - add operator
-        Mockito.verify(userDao, Mockito.times(1)).updateUser(Mockito.argThat(matchesPendingUser3));
-        Mockito.verify(userDao, Mockito.never()).createUser(Mockito.argThat(matchesUser3));
-        Mockito.verify(authorityDao, Mockito.times(1)).createAuthorities(EnumSet.of(Authority.ROLE_PROXY), USER_ID_3);
-        Mockito.verify(authorityDao, Mockito.never()).deleteAuthorities(Mockito.anySet(), Mockito.eq(USER_ID_3));
-
-        // matcher for user 4
-        Matcher<NiFiUser> matchesUser4 = new ArgumentMatcher<NiFiUser>() {
-            @Override
-            public boolean matches(Object argument) {
-                NiFiUser user = (NiFiUser) argument;
-                return USER_ID_4.equals(user.getId());
-            }
-        };
-
-        // verify user 4 - new user - add monitor
-        Mockito.verify(userDao, Mockito.never()).updateUser(Mockito.argThat(matchesUser4));
-        Mockito.verify(userDao, Mockito.times(1)).createUser(Mockito.argThat(matchesUser4));
-        Mockito.verify(authorityDao, Mockito.times(1)).createAuthorities(EnumSet.of(Authority.ROLE_MONITOR), USER_ID_4);
-        Mockito.verify(authorityDao, Mockito.never()).deleteAuthorities(Mockito.anySet(), Mockito.eq(USER_ID_4));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/SetUserAuthoritiesActionTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/SetUserAuthoritiesActionTest.java b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/SetUserAuthoritiesActionTest.java
deleted file mode 100644
index dd3695c..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/SetUserAuthoritiesActionTest.java
+++ /dev/null
@@ -1,223 +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.admin.service.action;
-
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.Set;
-import org.apache.nifi.admin.dao.AuthorityDAO;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.authorization.Authority;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.exception.AuthorityAccessException;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUser;
-import org.apache.commons.lang3.StringUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Test case for SetUserAuthoritiesAction.
- */
-public class SetUserAuthoritiesActionTest {
-
-    private static final String USER_ID_1 = "1";
-    private static final String USER_ID_2 = "2";
-    private static final String USER_ID_3 = "3";
-
-    private static final String USER_DN_2 = "user dn 2";
-    private static final String USER_DN_3 = "user dn 3";
-
-    private DAOFactory daoFactory;
-    private UserDAO userDao;
-    private AuthorityDAO authorityDao;
-    private AuthorityProvider authorityProvider;
-
-    @Before
-    public void setup() throws Exception {
-        // mock the user dao
-        userDao = Mockito.mock(UserDAO.class);
-        Mockito.doAnswer(new Answer<NiFiUser>() {
-            @Override
-            public NiFiUser answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String id = (String) args[0];
-
-                NiFiUser user = null;
-                if (USER_ID_1.equals(id)) {
-                    // leave user uninitialized
-                } else if (USER_ID_2.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_2);
-                    user.setDn(USER_DN_2);
-                } else if (USER_ID_3.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_3);
-                    user.setDn(USER_DN_3);
-                    user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
-                    user.setStatus(AccountStatus.ACTIVE);
-                }
-                return user;
-            }
-        }).when(userDao).findUserById(Mockito.anyString());
-        Mockito.doAnswer(new Answer<NiFiUser>() {
-            @Override
-            public NiFiUser answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String dn = (String) args[0];
-
-                NiFiUser user = null;
-                if (USER_DN_3.equals(dn)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_3);
-                    user.setDn(USER_DN_3);
-                    user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
-                    user.setStatus(AccountStatus.ACTIVE);
-                }
-                return user;
-            }
-        }).when(userDao).findUserByDn(Mockito.anyString());
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                NiFiUser user = (NiFiUser) args[0];
-
-                // do nothing
-                return null;
-            }
-        }).when(userDao).updateUser(Mockito.any(NiFiUser.class));
-
-        // mock the authority dao
-        authorityDao = Mockito.mock(AuthorityDAO.class);
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                Set<Authority> authorities = (Set<Authority>) args[0];
-                String id = (String) args[1];
-
-                // do nothing
-                return null;
-            }
-        }).when(authorityDao).createAuthorities(Mockito.anySetOf(Authority.class), Mockito.anyString());
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                Set<Authority> authorities = (Set<Authority>) args[0];
-                String id = (String) args[1];
-
-                // do nothing
-                return null;
-            }
-        }).when(authorityDao).deleteAuthorities(Mockito.anySetOf(Authority.class), Mockito.anyString());
-
-        // mock the dao factory
-        daoFactory = Mockito.mock(DAOFactory.class);
-        Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao);
-        Mockito.when(daoFactory.getAuthorityDAO()).thenReturn(authorityDao);
-
-        // mock the authority provider
-        authorityProvider = Mockito.mock(AuthorityProvider.class);
-        Mockito.doAnswer(new Answer<Set<Authority>>() {
-            @Override
-            public Set<Authority> answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String dn = (String) args[0];
-
-                Set<Authority> authorities = EnumSet.noneOf(Authority.class);
-                if (USER_DN_3.equals(dn)) {
-                    authorities.add(Authority.ROLE_DFM);
-                }
-
-                return authorities;
-            }
-        }).when(authorityProvider).getAuthorities(Mockito.anyString());
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String dn = (String) args[0];
-                Set<Authority> authorites = (Set<Authority>) args[1];
-
-                if (USER_DN_2.equals(dn)) {
-                    throw new AuthorityAccessException(StringUtils.EMPTY);
-                }
-
-                // do nothing
-                return null;
-            }
-        }).when(authorityProvider).setAuthorities(Mockito.anyString(), Mockito.anySet());
-    }
-
-    /**
-     * Test activating an unknown user account. User accounts are unknown then
-     * there is no pending account for the user.
-     *
-     * @throws Exception
-     */
-    @Test(expected = AccountNotFoundException.class)
-    public void testUnknownUser() throws Exception {
-        UpdateUserAction setUserAuthorities = new UpdateUserAction(USER_ID_1, Collections.EMPTY_SET);
-        setUserAuthorities.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Testing case then an AuthorityAccessException occurs while setting a
-     * users authorities.
-     *
-     * @throws Exception
-     */
-    @Test(expected = AdministrationException.class)
-    public void testAuthorityAccessException() throws Exception {
-        UpdateUserAction setUserAuthorities = new UpdateUserAction(USER_ID_2, Collections.EMPTY_SET);
-        setUserAuthorities.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Tests general case of setting user authorities.
-     *
-     * @throws Exception
-     */
-    @Test
-    public void testSetAuthorities() throws Exception {
-        UpdateUserAction setUserAuthorities = new UpdateUserAction(USER_ID_3, EnumSet.of(Authority.ROLE_ADMIN));
-        NiFiUser user = setUserAuthorities.execute(daoFactory, authorityProvider);
-
-        // verify user
-        Assert.assertEquals(USER_ID_3, user.getId());
-        Assert.assertEquals(1, user.getAuthorities().size());
-        Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_ADMIN));
-
-        // verify interaction with dao
-        Mockito.verify(userDao, Mockito.times(1)).updateUser(user);
-        Mockito.verify(authorityDao, Mockito.times(1)).createAuthorities(EnumSet.of(Authority.ROLE_ADMIN), USER_ID_3);
-
-        Set<Authority> authoritiesAddedToProvider = EnumSet.of(Authority.ROLE_ADMIN);
-
-        // verify interaction with provider
-        Mockito.verify(authorityProvider, Mockito.times(1)).setAuthorities(USER_DN_3, authoritiesAddedToProvider);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/.gitignore
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/.gitignore b/nar-bundles/framework-bundle/framework/client-dto/.gitignore
deleted file mode 100755
index cd1a4e7..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/.gitignore
+++ /dev/null
@@ -1,6 +0,0 @@
-/target
-/target
-/target
-/target
-/target
-/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/pom.xml b/nar-bundles/framework-bundle/framework/client-dto/pom.xml
deleted file mode 100644
index 924e83a..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/pom.xml
+++ /dev/null
@@ -1,46 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-framework-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-
-    <artifactId>client-dto</artifactId>
-    <name>NiFi Client Dto</name>
-    <build>
-        <plugins>
-            <!--
-                Always attach sources so the enunciate documentation
-                is complete.
-            -->
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-source-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <id>attach-sources</id>
-                        <goals>
-                            <goal>jar-no-fork</goal>
-                        </goals>
-                    </execution>
-                </executions>
-            </plugin>
-        </plugins>
-    </build>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/AboutDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/AboutDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/AboutDTO.java
deleted file mode 100644
index 0e2dcb0..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/AboutDTO.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.api.dto;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Contains details about this NiFi including the title and version.
- */
-@XmlType(name = "about")
-public class AboutDTO {
-
-    private String title;
-    private String version;
-
-    /* getters / setters */
-    /**
-     * The title to be used on the page and in the About dialog.
-     *
-     * @return The title
-     */
-    public String getTitle() {
-        return title;
-    }
-
-    public void setTitle(String title) {
-        this.title = title;
-    }
-
-    /**
-     * The version of this NiFi.
-     *
-     * @return The version.
-     */
-    public String getVersion() {
-        return version;
-    }
-
-    public void setVersion(String version) {
-        this.version = version;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BannerDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BannerDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BannerDTO.java
deleted file mode 100644
index 70c408b..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BannerDTO.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.api.dto;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Banners that should appear on the top and bottom of this NiFi.
- */
-@XmlType(name = "banners")
-public class BannerDTO {
-
-    private String headerText;
-    private String footerText;
-
-    /* getters / setters */
-    /**
-     * The banner footer text.
-     *
-     * @return The footer text
-     */
-    public String getFooterText() {
-        return footerText;
-    }
-
-    public void setFooterText(String footerText) {
-        this.footerText = footerText;
-    }
-
-    /**
-     * The banner header text.
-     *
-     * @return The header text
-     */
-    public String getHeaderText() {
-        return headerText;
-    }
-
-    public void setHeaderText(String headerText) {
-        this.headerText = headerText;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinBoardDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinBoardDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinBoardDTO.java
deleted file mode 100644
index ddc3d2e..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinBoardDTO.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.web.api.dto;
-
-import java.util.Date;
-import java.util.List;
-
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * The contents for the bulletin board including the bulletins and the timestamp
- * when the board was generated.
- */
-@XmlType(name = "bulletinBoard")
-public class BulletinBoardDTO {
-
-    private List<BulletinDTO> bulletins;
-    private Date generated;
-
-    /**
-     * The bulletins to populate in the bulletin board.
-     *
-     * @return
-     */
-    public List<BulletinDTO> getBulletins() {
-        return bulletins;
-    }
-
-    public void setBulletins(List<BulletinDTO> bulletins) {
-        this.bulletins = bulletins;
-    }
-
-    /**
-     * When this bulletin board was generated.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    public Date getGenerated() {
-        return generated;
-    }
-
-    public void setGenerated(final Date generated) {
-        this.generated = generated;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinDTO.java
deleted file mode 100644
index c6aca24..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinDTO.java
+++ /dev/null
@@ -1,161 +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.api.dto;
-
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * A bulletin that represents a notification about a passing event including,
- * the source component (if applicable), the timestamp, the message, and where
- * the bulletin originated (if applicable).
- */
-@XmlType(name = "bulletin")
-public class BulletinDTO {
-
-    private Long id;
-    private String nodeAddress;
-    private String category;
-    private String groupId;
-    private String sourceId;
-    private String sourceName;
-    private String level;
-    private String message;
-    private Date timestamp;
-
-    /**
-     * The id of this message.
-     *
-     * @return
-     */
-    public Long getId() {
-        return id;
-    }
-
-    public void setId(Long id) {
-        this.id = id;
-    }
-
-    /**
-     * When clustered, the address of the node from which this bulletin
-     * originated.
-     *
-     * @return
-     */
-    public String getNodeAddress() {
-        return nodeAddress;
-    }
-
-    public void setNodeAddress(String nodeAddress) {
-        this.nodeAddress = nodeAddress;
-    }
-
-    /**
-     * The group id of the source component.
-     *
-     * @return
-     */
-    public String getGroupId() {
-        return groupId;
-    }
-
-    public void setGroupId(String groupId) {
-        this.groupId = groupId;
-    }
-
-    /**
-     * The category of this message.
-     *
-     * @return
-     */
-    public String getCategory() {
-        return category;
-    }
-
-    public void setCategory(String category) {
-        this.category = category;
-    }
-
-    /**
-     * The actual message.
-     *
-     * @return
-     */
-    public String getMessage() {
-        return message;
-    }
-
-    public void setMessage(String message) {
-        this.message = message;
-    }
-
-    /**
-     * The id of the source of this message.
-     *
-     * @return
-     */
-    public String getSourceId() {
-        return sourceId;
-    }
-
-    public void setSourceId(String sourceId) {
-        this.sourceId = sourceId;
-    }
-
-    /**
-     * The name of the source of this message.
-     *
-     * @return
-     */
-    public String getSourceName() {
-        return sourceName;
-    }
-
-    public void setSourceName(String sourceName) {
-        this.sourceName = sourceName;
-    }
-
-    /**
-     * The level of this bulletin.
-     *
-     * @return
-     */
-    public String getLevel() {
-        return level;
-    }
-
-    public void setLevel(String level) {
-        this.level = level;
-    }
-
-    /**
-     * When this bulletin was generated as a formatted string.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    public Date getTimestamp() {
-        return timestamp;
-    }
-
-    public void setTimestamp(Date timestamp) {
-        this.timestamp = timestamp;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinQueryDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinQueryDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinQueryDTO.java
deleted file mode 100644
index 015b174..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinQueryDTO.java
+++ /dev/null
@@ -1,113 +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.api.dto;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * A query for bulletin board. Will filter the resulting bulletin board
- * according to the criteria in this query.
- */
-@XmlType(name = "bulletinQuery")
-public class BulletinQueryDTO {
-
-    private String sourceId;
-    private String groupId;
-    private String name;
-    private String message;
-    private Long after;
-    private Integer limit;
-
-    /**
-     * Include bulletins after this id.
-     *
-     * @return
-     */
-    public Long getAfter() {
-        return after;
-    }
-
-    public void setAfter(Long after) {
-        this.after = after;
-    }
-
-    /**
-     * Include bulletin within this group. Supports a regular expression.
-     *
-     * @return
-     */
-    public String getGroupId() {
-        return groupId;
-    }
-
-    public void setGroupId(String groupId) {
-        this.groupId = groupId;
-    }
-
-    /**
-     * Include bulletins that match this message. Supports a regular expression.
-     *
-     * @return
-     */
-    public String getMessage() {
-        return message;
-    }
-
-    public void setMessage(String message) {
-        this.message = message;
-    }
-
-    /**
-     * Include bulletins that match this name. Supports a regular expression.
-     *
-     * @return
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * Include bulletins that match this id. Supports a source id.
-     *
-     * @return
-     */
-    public String getSourceId() {
-        return sourceId;
-    }
-
-    public void setSourceId(String sourceId) {
-        this.sourceId = sourceId;
-    }
-
-    /**
-     * The maximum number of bulletins to return.
-     *
-     * @return
-     */
-    public Integer getLimit() {
-        return limit;
-    }
-
-    public void setLimit(Integer limit) {
-        this.limit = limit;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ClusterDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ClusterDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ClusterDTO.java
deleted file mode 100644
index 53100e3..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ClusterDTO.java
+++ /dev/null
@@ -1,61 +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.api.dto;
-
-import java.util.Collection;
-import java.util.Date;
-
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * Details about the composition of the cluster at a specific date/time.
- */
-@XmlType(name = "cluster")
-public class ClusterDTO {
-
-    private Collection<NodeDTO> nodes;
-    private Date generated;
-
-    /**
-     * The collection of the node DTOs.
-     *
-     * @return
-     */
-    public Collection<NodeDTO> getNodes() {
-        return nodes;
-    }
-
-    public void setNodes(Collection<NodeDTO> nodes) {
-        this.nodes = nodes;
-    }
-
-    /**
-     * Gets the date/time that this report was generated.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    public Date getGenerated() {
-        return generated;
-    }
-
-    public void setGenerated(Date generated) {
-        this.generated = generated;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectableDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectableDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectableDTO.java
deleted file mode 100644
index 1be480c..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectableDTO.java
+++ /dev/null
@@ -1,146 +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.api.dto;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Details about a connectable component.
- */
-@XmlType(name = "connectable")
-public class ConnectableDTO {
-
-    private String id;
-    private String type;
-    private String groupId;
-    private String name;
-    private Boolean running;
-    private Boolean transmitting;
-    private Boolean exists;
-    private String comments;
-
-    /**
-     * The id of this connectable component.
-     *
-     * @return
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The type of this connectable component.
-     *
-     * @return
-     */
-    public String getType() {
-        return type;
-    }
-
-    public void setType(String type) {
-        this.type = type;
-    }
-
-    /**
-     * The id of the group that this connectable component resides in.
-     *
-     * @return
-     */
-    public String getGroupId() {
-        return groupId;
-    }
-
-    public void setGroupId(String groupId) {
-        this.groupId = groupId;
-    }
-
-    /**
-     * The name of this connectable component.
-     *
-     * @return
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * Used to reflect the current state of this Connectable.
-     *
-     * @return
-     */
-    public Boolean isRunning() {
-        return running;
-    }
-
-    public void setRunning(Boolean running) {
-        this.running = running;
-    }
-
-    /**
-     * If this represents a remote port it is used to indicate whether the
-     * target exists.
-     *
-     * @return
-     */
-    public Boolean getExists() {
-        return exists;
-    }
-
-    public void setExists(Boolean exists) {
-        this.exists = exists;
-    }
-
-    /**
-     * If this represents a remote port it is used to indicate whether is it
-     * configured to transmit.
-     *
-     * @return
-     */
-    public Boolean getTransmitting() {
-        return transmitting;
-    }
-
-    public void setTransmitting(Boolean transmitting) {
-        this.transmitting = transmitting;
-    }
-
-    /**
-     * The comments from this Connectable.
-     *
-     * @return
-     */
-    public String getComments() {
-        return comments;
-    }
-
-    public void setComments(String comments) {
-        this.comments = comments;
-    }
-
-    @Override
-    public String toString() {
-        return "ConnectableDTO [Type=" + type + ", Name=" + name + ", Id=" + id + "]";
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectionDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectionDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectionDTO.java
deleted file mode 100644
index 660820c..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectionDTO.java
+++ /dev/null
@@ -1,215 +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.api.dto;
-
-import java.util.List;
-import java.util.Set;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * A connection between two connectable components.
- */
-@XmlType(name = "connection")
-public class ConnectionDTO extends NiFiComponentDTO {
-
-    private ConnectableDTO source;
-    private ConnectableDTO destination;
-    private String name;
-    private Integer labelIndex;
-    private Long zIndex;
-    private Set<String> selectedRelationships;
-    private Set<String> availableRelationships;
-
-    private Long backPressureObjectThreshold;
-    private String backPressureDataSizeThreshold;
-    private String flowFileExpiration;
-    private List<String> prioritizers;
-    private List<PositionDTO> bends;
-
-    /**
-     * The id of the source processor.
-     *
-     * @return The id of the source processor
-     */
-    public ConnectableDTO getSource() {
-        return source;
-    }
-
-    public void setSource(ConnectableDTO source) {
-        this.source = source;
-    }
-
-    /**
-     * The id of the target processor.
-     *
-     * @return The id of the target processor
-     */
-    public ConnectableDTO getDestination() {
-        return destination;
-    }
-
-    public void setDestination(ConnectableDTO destination) {
-        this.destination = destination;
-    }
-
-    /**
-     * The name of the connection.
-     *
-     * @return
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * The position of the bend points on this connection.
-     *
-     * @return
-     */
-    public List<PositionDTO> getBends() {
-        return bends;
-    }
-
-    public void setBends(List<PositionDTO> bends) {
-        this.bends = bends;
-    }
-
-    /**
-     * The index of control point that the connection label should be placed
-     * over.
-     *
-     * @return
-     */
-    public Integer getLabelIndex() {
-        return labelIndex;
-    }
-
-    public void setLabelIndex(Integer labelIndex) {
-        this.labelIndex = labelIndex;
-    }
-
-    /**
-     * The z index for this connection.
-     *
-     * @return
-     */
-    public Long getzIndex() {
-        return zIndex;
-    }
-
-    public void setzIndex(Long zIndex) {
-        this.zIndex = zIndex;
-    }
-
-    /**
-     * The relationships that make up this connection.
-     *
-     * @return The relationships
-     */
-    public Set<String> getSelectedRelationships() {
-        return selectedRelationships;
-    }
-
-    public void setSelectedRelationships(Set<String> relationships) {
-        this.selectedRelationships = relationships;
-    }
-
-    /**
-     * The relationships that the source of the connection currently supports.
-     * This property is read only.
-     *
-     * @return
-     */
-    public Set<String> getAvailableRelationships() {
-        return availableRelationships;
-    }
-
-    public void setAvailableRelationships(Set<String> availableRelationships) {
-        this.availableRelationships = availableRelationships;
-    }
-
-    /**
-     * The object count threshold for determining when back pressure is applied.
-     * Updating this value is a passive change in the sense that it won't impact
-     * whether existing files over the limit are affected but it does help
-     * feeder processors to stop pushing too much into this work queue.
-     *
-     * @return The back pressure object threshold
-     */
-    public Long getBackPressureObjectThreshold() {
-        return backPressureObjectThreshold;
-    }
-
-    public void setBackPressureObjectThreshold(Long backPressureObjectThreshold) {
-        this.backPressureObjectThreshold = backPressureObjectThreshold;
-    }
-
-    /**
-     * The object data size threshold for determining when back pressure is
-     * applied. Updating this value is a passive change in the sense that it
-     * won't impact whether existing files over the limit are affected but it
-     * does help feeder processors to stop pushing too much into this work
-     * queue.
-     *
-     * @return The back pressure data size threshold
-     */
-    public String getBackPressureDataSizeThreshold() {
-        return backPressureDataSizeThreshold;
-    }
-
-    public void setBackPressureDataSizeThreshold(String backPressureDataSizeThreshold) {
-        this.backPressureDataSizeThreshold = backPressureDataSizeThreshold;
-    }
-
-    /**
-     * The amount of time a flow file may be in the flow before it will be
-     * automatically aged out of the flow. Once a flow file reaches this age it
-     * will be terminated from the flow the next time a processor attempts to
-     * start work on it.
-     *
-     * @return The flow file expiration in minutes
-     */
-    public String getFlowFileExpiration() {
-        return flowFileExpiration;
-    }
-
-    public void setFlowFileExpiration(String flowFileExpiration) {
-        this.flowFileExpiration = flowFileExpiration;
-    }
-
-    /**
-     * The prioritizers this processor is using.
-     *
-     * @return The prioritizer list
-     */
-    public List<String> getPrioritizers() {
-        return prioritizers;
-    }
-
-    public void setPrioritizers(List<String> prioritizers) {
-        this.prioritizers = prioritizers;
-    }
-
-    @Override
-    public String toString() {
-        return "ConnectionDTO [name: " + name + " from " + source + " to " + destination + "]";
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.java
deleted file mode 100644
index b916025..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.java
+++ /dev/null
@@ -1,158 +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.api.dto;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Details for the controller configuration.
- */
-@XmlType(name = "config")
-public class ControllerConfigurationDTO {
-
-    private String name;
-    private String comments;
-    private Integer maxTimerDrivenThreadCount;
-    private Integer maxEventDrivenThreadCount;
-
-    private Long autoRefreshIntervalSeconds;
-    private Boolean siteToSiteSecure;
-
-    private Integer timeOffset;
-
-    private String contentViewerUrl;
-    private String uri;
-
-    /**
-     * The maximum number of timer driven threads this NiFi has available.
-     *
-     * @return The maximum number of threads
-     */
-    public Integer getMaxTimerDrivenThreadCount() {
-        return maxTimerDrivenThreadCount;
-    }
-
-    public void setMaxTimerDrivenThreadCount(Integer maxTimerDrivenThreadCount) {
-        this.maxTimerDrivenThreadCount = maxTimerDrivenThreadCount;
-    }
-
-    /**
-     * The maximum number of event driven thread this NiFi has available.
-     *
-     * @return
-     */
-    public Integer getMaxEventDrivenThreadCount() {
-        return maxEventDrivenThreadCount;
-    }
-
-    public void setMaxEventDrivenThreadCount(Integer maxEventDrivenThreadCount) {
-        this.maxEventDrivenThreadCount = maxEventDrivenThreadCount;
-    }
-
-    /**
-     * The name of this NiFi.
-     *
-     * @return The name
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * The comments for this NiFi.
-     *
-     * @return
-     */
-    public String getComments() {
-        return comments;
-    }
-
-    public void setComments(String comments) {
-        this.comments = comments;
-    }
-
-    /**
-     * The interval in seconds between the automatic NiFi refresh requests. This
-     * value is read only.
-     *
-     * @return The interval in seconds
-     */
-    public Long getAutoRefreshIntervalSeconds() {
-        return autoRefreshIntervalSeconds;
-    }
-
-    public void setAutoRefreshIntervalSeconds(Long autoRefreshIntervalSeconds) {
-        this.autoRefreshIntervalSeconds = autoRefreshIntervalSeconds;
-    }
-
-    /**
-     * Indicates whether or not Site-to-Site communications with this instance
-     * is secure (2-way authentication). This value is read only.
-     *
-     * @return
-     */
-    public Boolean isSiteToSiteSecure() {
-        return siteToSiteSecure;
-    }
-
-    public void setSiteToSiteSecure(Boolean siteToSiteSecure) {
-        this.siteToSiteSecure = siteToSiteSecure;
-    }
-
-    /**
-     * The time offset of the server.
-     *
-     * @return
-     */
-    public Integer getTimeOffset() {
-        return timeOffset;
-    }
-
-    public void setTimeOffset(Integer timeOffset) {
-        this.timeOffset = timeOffset;
-    }
-
-    /**
-     * Returns the URL for the content viewer if configured.
-     *
-     * @return
-     */
-    public String getContentViewerUrl() {
-        return contentViewerUrl;
-    }
-
-    public void setContentViewerUrl(String contentViewerUrl) {
-        this.contentViewerUrl = contentViewerUrl;
-    }
-
-    /**
-     * The URI for this NiFi controller.
-     *
-     * @return
-     */
-    public String getUri() {
-        return uri;
-    }
-
-    public void setUri(String uri) {
-        this.uri = uri;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerDTO.java
deleted file mode 100644
index 9e15fc1..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerDTO.java
+++ /dev/null
@@ -1,262 +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.api.dto;
-
-import java.util.Set;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Configuration details for a NiFi controller. Primary use of this DTO is for
- * consumption by a remote NiFi instance to initiate site to site
- * communications.
- */
-@XmlType(name = "controller")
-public class ControllerDTO {
-
-    private String id;
-    private String name;
-    private String comments;
-
-    private Integer runningCount;
-    private Integer stoppedCount;
-    private Integer invalidCount;
-    private Integer disabledCount;
-    private Integer activeRemotePortCount;
-    private Integer inactiveRemotePortCount;
-
-    private Integer inputPortCount;
-    private Integer outputPortCount;
-
-    private Integer remoteSiteListeningPort;
-    private Boolean siteToSiteSecure;
-    private String instanceId;
-    private Set<PortDTO> inputPorts;
-    private Set<PortDTO> outputPorts;
-
-    /**
-     * The id of this NiFi controller.
-     *
-     * @return
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The name of this NiFi controller.
-     *
-     * @return The name of this controller
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * The comments of this NiFi controller.
-     *
-     * @return
-     */
-    public String getComments() {
-        return comments;
-    }
-
-    public void setComments(String comments) {
-        this.comments = comments;
-    }
-
-    /**
-     * The input ports available to send data to this NiFi controller.
-     *
-     * @return
-     */
-    public Set<PortDTO> getInputPorts() {
-        return inputPorts;
-    }
-
-    public void setInputPorts(Set<PortDTO> inputPorts) {
-        this.inputPorts = inputPorts;
-    }
-
-    /**
-     * The output ports available to received data from this NiFi controller.
-     *
-     * @return
-     */
-    public Set<PortDTO> getOutputPorts() {
-        return outputPorts;
-    }
-
-    public void setOutputPorts(Set<PortDTO> outputPorts) {
-        this.outputPorts = outputPorts;
-    }
-
-    /**
-     * The Instance ID of the cluster, if this node is connected to a Cluster
-     * Manager, or of this individual instance of in standalone mode
-     *
-     * @return
-     */
-    public String getInstanceId() {
-        return instanceId;
-    }
-
-    public void setInstanceId(String instanceId) {
-        this.instanceId = instanceId;
-    }
-
-    /**
-     * The Socket Port on which this instance is listening for Remote Transfers
-     * of Flow Files. If this instance is not configured to receive Flow Files
-     * from remote instances, this will be null.
-     *
-     * @return a integer between 1 and 65535, or null, if not configured for
-     * remote transfer
-     */
-    public Integer getRemoteSiteListeningPort() {
-        return remoteSiteListeningPort;
-    }
-
-    public void setRemoteSiteListeningPort(final Integer port) {
-        this.remoteSiteListeningPort = port;
-    }
-
-    /**
-     * Indicates whether or not Site-to-Site communications with this instance
-     * is secure (2-way authentication)
-     *
-     * @return
-     */
-    public Boolean isSiteToSiteSecure() {
-        return siteToSiteSecure;
-    }
-
-    public void setSiteToSiteSecure(Boolean siteToSiteSecure) {
-        this.siteToSiteSecure = siteToSiteSecure;
-    }
-
-    /**
-     * The number of running components in this process group.
-     *
-     * @return
-     */
-    public Integer getRunningCount() {
-        return runningCount;
-    }
-
-    public void setRunningCount(Integer runningCount) {
-        this.runningCount = runningCount;
-    }
-
-    /**
-     * The number of stopped components in this process group.
-     *
-     * @return
-     */
-    public Integer getStoppedCount() {
-        return stoppedCount;
-    }
-
-    public void setStoppedCount(Integer stoppedCount) {
-        this.stoppedCount = stoppedCount;
-    }
-
-    /**
-     * The number of active remote ports contained in this process group.
-     *
-     * @return
-     */
-    public Integer getActiveRemotePortCount() {
-        return activeRemotePortCount;
-    }
-
-    public void setActiveRemotePortCount(Integer activeRemotePortCount) {
-        this.activeRemotePortCount = activeRemotePortCount;
-    }
-
-    /**
-     * The number of inactive remote ports contained in this process group.
-     *
-     * @return
-     */
-    public Integer getInactiveRemotePortCount() {
-        return inactiveRemotePortCount;
-    }
-
-    public void setInactiveRemotePortCount(Integer inactiveRemotePortCount) {
-        this.inactiveRemotePortCount = inactiveRemotePortCount;
-    }
-
-    /**
-     * The number of input ports contained in this process group.
-     *
-     * @return
-     */
-    public Integer getInputPortCount() {
-        return inputPortCount;
-    }
-
-    public void setInputPortCount(Integer inputPortCount) {
-        this.inputPortCount = inputPortCount;
-    }
-
-    /**
-     * The number of invalid components in this process group.
-     *
-     * @return
-     */
-    public Integer getInvalidCount() {
-        return invalidCount;
-    }
-
-    public void setInvalidCount(Integer invalidCount) {
-        this.invalidCount = invalidCount;
-    }
-
-    /**
-     * The number of disabled components in this process group.
-     *
-     * @return
-     */
-    public Integer getDisabledCount() {
-        return disabledCount;
-    }
-
-    public void setDisabledCount(Integer disabledCount) {
-        this.disabledCount = disabledCount;
-    }
-
-    /**
-     * The number of output ports in this process group.
-     *
-     * @return
-     */
-    public Integer getOutputPortCount() {
-        return outputPortCount;
-    }
-
-    public void setOutputPortCount(Integer outputPortCount) {
-        this.outputPortCount = outputPortCount;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/CounterDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/CounterDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/CounterDTO.java
deleted file mode 100644
index 10ea41d..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/CounterDTO.java
+++ /dev/null
@@ -1,94 +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.api.dto;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Counter value for a specific component in a specific context. A counter is a
- * value that a component can adjust during processing.
- */
-@XmlType(name = "counter")
-public class CounterDTO {
-
-    private String id;
-    private String context;
-    private String name;
-    private Long valueCount;
-    private String value;
-
-    /**
-     * The context of the counter.
-     *
-     * @return
-     */
-    public String getContext() {
-        return context;
-    }
-
-    public void setContext(String context) {
-        this.context = context;
-    }
-
-    /**
-     * The id of the counter.
-     *
-     * @return
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The name of the counter
-     *
-     * @return
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * The value for the counter
-     *
-     * @return
-     */
-    public String getValue() {
-        return value;
-    }
-
-    public void setValue(String value) {
-        this.value = value;
-    }
-
-    public Long getValueCount() {
-        return valueCount;
-    }
-
-    public void setValueCount(Long valueCount) {
-        this.valueCount = valueCount;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/CountersDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/CountersDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/CountersDTO.java
deleted file mode 100644
index ac1aa38..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/CountersDTO.java
+++ /dev/null
@@ -1,61 +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.api.dto;
-
-import java.util.Collection;
-import java.util.Date;
-
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * All the counters in this NiFi instance at a given time.
- */
-@XmlType(name = "counters")
-public class CountersDTO {
-
-    private Date generated;
-    private Collection<CounterDTO> counters;
-
-    /**
-     * Gets the collection of counters.
-     *
-     * @return
-     */
-    public Collection<CounterDTO> getCounters() {
-        return counters;
-    }
-
-    public void setCounters(Collection<CounterDTO> counters) {
-        this.counters = counters;
-    }
-
-    /**
-     * Gets the date/time that this report was generated.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    public Date getGenerated() {
-        return generated;
-    }
-
-    public void setGenerated(Date generated) {
-        this.generated = generated;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.java
deleted file mode 100644
index 5a2d789..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.java
+++ /dev/null
@@ -1,71 +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.api.dto;
-
-import java.util.Set;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Class used for providing documentation of a specified type that may be
- * instantiated.
- */
-@XmlType(name = "documentedType")
-public class DocumentedTypeDTO {
-
-    private String type;
-    private String description;
-    private Set<String> tags;
-
-    /**
-     * An optional description of the corresponding type.
-     *
-     * @return
-     */
-    public String getDescription() {
-        return description;
-    }
-
-    public void setDescription(String description) {
-        this.description = description;
-    }
-
-    /**
-     * The type is the fully-qualified name of a Java class.
-     *
-     * @return
-     */
-    public String getType() {
-        return type;
-    }
-
-    public void setType(String type) {
-        this.type = type;
-    }
-
-    /**
-     * The tags associated with this type
-     *
-     * @return
-     */
-    public Set<String> getTags() {
-        return tags;
-    }
-
-    public void setTags(final Set<String> tags) {
-        this.tags = tags;
-    }
-}


[43/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java b/commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
deleted file mode 100644
index 8dfbaf1..0000000
--- a/commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
+++ /dev/null
@@ -1,1143 +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.attribute.expression.language;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.text.SimpleDateFormat;
-import java.util.Calendar;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.Query.Range;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
-import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-import org.apache.nifi.flowfile.FlowFile;
-import org.antlr.runtime.tree.Tree;
-import org.junit.Assert;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-public class TestQuery {
-
-    @Test
-    public void testCompilation() {
-        assertInvalid("${attr:uuid()}");
-        assertInvalid("${attr:indexOf(length())}");
-        assertValid("${UUID()}");
-        assertInvalid("${UUID():nextInt()}");
-        assertValid("${nextInt()}");
-        assertValid("${now():format('yyyy/MM/dd')}");
-        assertInvalid("${attr:times(3)}");
-        assertValid("${attr:toNumber():multiply(3)}");
-        // left here because it's convenient for looking at the output
-        //System.out.println(Query.compile("").evaluate(null));
-    }
-    
-    private void assertValid(final String query) {
-        try {
-            Query.compile(query);
-        } catch (final Exception e) {
-            e.printStackTrace();
-            Assert.fail("Expected query to be valid, but it failed to compile due to " + e);
-        }
-    }
-    
-    private void assertInvalid(final String query) {
-        try {
-            Query.compile(query);
-            Assert.fail("Expected query to be invalid, but it did compile");
-        } catch (final Exception e) {
-        }
-    }
-    
-    @Test
-    public void testIsValidExpression() {
-        Query.validateExpression("${abc:substring(${xyz:length()})}", false);
-        Query.isValidExpression("${now():format('yyyy-MM-dd')}");
-        
-        
-        try {
-            Query.validateExpression("$${attr}", false);
-            Assert.fail("invalid query validated");
-        } catch (final AttributeExpressionLanguageParsingException e) {
-        }
-        
-        Query.validateExpression("$${attr}", true);
-        
-        Query.validateExpression("${filename:startsWith('T8MTXBC')\n" 
-            + ":or( ${filename:startsWith('C4QXABC')} )\n"
-            + ":or( ${filename:startsWith('U6CXEBC')} )"
-            + ":or( ${filename:startsWith('KYM3ABC')} )}", false);
-    }
-
-    
-    @Test
-    public void testCompileEmbedded() {
-        final String expression = "${x:equals( ${y} )}";
-        final Query query = Query.compile(expression);
-        final Tree tree = query.getTree();
-        System.out.println( printTree(tree) );
-        
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("x", "x");
-        attributes.put("y", "x");
-        final String result = Query.evaluateExpressions(expression, attributes, null);
-        assertEquals("true", result);
-        
-        Query.validateExpression(expression, false);
-    }
-    
-    private String printTree(final Tree tree) {
-        final StringBuilder sb = new StringBuilder();
-        printTree(tree, 0, sb);
-        
-        return sb.toString();
-    }
-    
-    private void printTree(final Tree tree, final int spaces, final StringBuilder sb) {
-        for (int i=0; i < spaces; i++) {
-            sb.append(" ");
-        }
-        
-        if ( tree.getText().trim().isEmpty() ) {
-            sb.append(tree.toString()).append("\n");
-        } else {
-            sb.append(tree.getText()).append("\n");
-        }
-        
-        for (int i=0; i < tree.getChildCount(); i++) {
-            printTree(tree.getChild(i), spaces + 2, sb);
-        }
-    }
-
-    @Test
-    public void testEscape() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("attr", "My Value");
-        attributes.put("${xx}", "hello");
-        
-        assertEquals("My Value", evaluateQueryForEscape("${attr}", attributes));
-        assertEquals("${attr}", evaluateQueryForEscape("$${attr}", attributes));
-        assertEquals("$My Value", evaluateQueryForEscape("$$${attr}", attributes));
-        assertEquals("$${attr}", evaluateQueryForEscape("$$$${attr}", attributes));
-        assertEquals("$$My Value", evaluateQueryForEscape("$$$$${attr}", attributes));
-    }
-
-    @Test
-    public void testWithBackSlashes() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("x", "C:\\test\\1.txt");
-        attributes.put("y", "y\ny");
-        
-        final String query = "${x:substringAfterLast( '/' ):substringAfterLast( '\\\\' )}";
-        verifyEquals(query, attributes, "1.txt");
-        attributes.put("x", "C:/test/1.txt");
-        verifyEquals(query, attributes, "1.txt");
-        
-        verifyEquals("${y:equals('y\\ny')}", attributes, Boolean.TRUE);
-    }
-    
-    @Test
-    public void testWithTicksOutside() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("attr", "My Value");
-
-        assertEquals(1, Query.extractExpressionRanges("\"${attr}").size());
-        assertEquals(1, Query.extractExpressionRanges("'${attr}").size());
-        assertEquals(1, Query.extractExpressionRanges("'${attr}'").size());
-        assertEquals(1, Query.extractExpressionRanges("${attr}").size());
-
-        assertEquals("'My Value'", Query.evaluateExpressions("'${attr}'", attributes, null));
-        assertEquals("'My Value", Query.evaluateExpressions("'${attr}", attributes, null));
-    }
-
-    
-    @Test
-    @Ignore("Depends on TimeZone")
-    public void testDateToNumber() {
-        final Query query = Query.compile("${dateTime:toDate('yyyy/MM/dd HH:mm:ss.SSS'):toNumber()}");
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("dateTime", "2013/11/18 10:22:27.678");
-        
-        final QueryResult<?> result = query.evaluate(attributes);
-        assertEquals(ResultType.NUMBER, result.getResultType());
-        assertEquals(1384788147678L, result.getValue());
-    }
-
-    @Test
-    public void testAddOneDayToDate() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("dateTime", "2013/11/18 10:22:27.678");
-
-        verifyEquals("${dateTime:toDate('yyyy/MM/dd HH:mm:ss.SSS'):toNumber():plus(86400000):toDate():format('yyyy/MM/dd HH:mm:ss.SSS')}", attributes, "2013/11/19 10:22:27.678");
-        verifyEquals("${dateTime:toDate('yyyy/MM/dd HH:mm:ss.SSS'):plus(86400000):format('yyyy/MM/dd HH:mm:ss.SSS')}", attributes, "2013/11/19 10:22:27.678");
-    }
-
-    @Test
-    @Ignore("Requires specific locale")
-    public void implicitDateConversion() {
-        final Date date = new Date();
-        final Query query = Query.compile("${dateTime:format('yyyy/MM/dd HH:mm:ss.SSS')}");
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("dateTime", date.toString());
-        
-        // the date.toString() above will end up truncating the milliseconds. So remove millis from the Date before
-        // formatting it
-        final SimpleDateFormat sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS", Locale.US);
-        final long millis = date.getTime() % 1000L;
-        final Date roundedToNearestSecond = new Date(date.getTime() - millis);
-        final String formatted = sdf.format(roundedToNearestSecond);
-        
-        final QueryResult<?> result = query.evaluate(attributes);
-        assertEquals(ResultType.STRING, result.getResultType());
-        assertEquals(formatted, result.getValue());
-    }
-
-    
-    @Test
-    public void testEmbeddedExpressionsAndQuotes() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("x", "abc");
-        attributes.put("a", "abc");
-        
-        verifyEquals("${x:equals(${a})}", attributes, true);
-        
-        Query.validateExpression("${x:equals('${a}')}", false);
-        assertEquals("true", Query.evaluateExpressions("${x:equals('${a}')}", attributes, null));
-        
-        Query.validateExpression("${x:equals(\"${a}\")}", false);
-        assertEquals("true", Query.evaluateExpressions("${x:equals(\"${a}\")}", attributes, null));
-    }
-    
-    @Test
-    public void testJoin() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("a.a", "a");
-        attributes.put("a.b", "b");
-        attributes.put("a.c", "c");
-        verifyEquals("${allAttributes( 'a.a', 'a.b', 'a.c' ):join(', ')}", attributes, "a, b, c");
-        verifyEquals("${x:join(', ')}", attributes, "");
-        verifyEquals("${a.a:join(', ')}", attributes, "a");
-        verifyEquals("${allAttributes( 'x', 'y' ):join(',')}", attributes, ",");
-    }
-    
-    @Test(expected=AttributeExpressionLanguageException.class)
-    public void testCannotCombineWithNonReducingFunction() {
-        Query.compileTree("${allAttributes( 'a.1' ):plus(1)}");
-    }
-
-
-    @Test
-    public void testIsEmpty() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("a", "a");
-        attributes.put("b", "");
-        attributes.put("c", "        \n");
-
-        verifyEquals("${a:isEmpty()}", attributes, false);
-        verifyEquals("${b:isEmpty()}", attributes, true);
-        verifyEquals("${c:isEmpty()}", attributes, true);
-        verifyEquals("${d:isEmpty()}", attributes, true);
-    }
-
-
-    @Test
-    public void testReplaceEmpty() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("a", "a");
-        attributes.put("b", "");
-        attributes.put("c", "        \n");
-            
-        verifyEquals("${a:replaceEmpty('c')}", attributes, "a");
-        verifyEquals("${b:replaceEmpty('c')}", attributes, "c");
-        verifyEquals("${c:replaceEmpty('c')}", attributes, "c");
-        verifyEquals("${d:replaceEmpty('c')}", attributes, "c");
-    }
-
-
-
-    @Test
-    public void testCount() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("a", "a");
-        attributes.put("b", "abc");
-        attributes.put("c", "        \n");
-        attributes.put("n1", "111");
-        attributes.put("n2", "222");
-        attributes.put("n3", "333333");
-
-        verifyEquals("${allMatchingAttributes( '.*' ):count()}", attributes, 6L);
-        verifyEquals("${allMatchingAttributes( '.*' ):length():gt(2):count()}", attributes, 5L);
-        verifyEquals("${allMatchingAttributes( 'n.*' ):plus(1):count()}", attributes, 3L );
-    }
-    
-    
-    @Test
-    public void testCurlyBracesInQuotes() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("attr", "My Valuee");
-        
-        assertEquals("Val", evaluateQueryForEscape("${attr:replaceAll('My (Val)ue{1,2}', '$1')}", attributes));
-        assertEquals("Val", evaluateQueryForEscape("${attr:replaceAll(\"My (Val)ue{1,2}\", '$1')}", attributes));
-    }
-    
-    
-    private String evaluateQueryForEscape(final String queryString, final Map<String, String> attributes) {
-        FlowFile mockFlowFile = Mockito.mock(FlowFile.class);
-        Mockito.when(mockFlowFile.getAttributes()).thenReturn(attributes);
-        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);
-    }
-    
-    
-    @Test
-    public void testGetAttributeValue() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("attr", "My Value");
-        verifyEquals("${attr}", attributes, "My Value");
-    }
-    
-    @Test
-    public void testGetAttributeValueEmbedded() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("attr", "XX ");
-        attributes.put("XX", "My Value");
-        verifyEquals("${${attr:trim()}}", attributes, "My Value");
-    }
-    
-    @Test
-    public void testSimpleSubstring() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("attr", "My Value");
-        verifyEquals("${attr:substring(2, 5)}", attributes, " Va");
-    }
-    
-    @Test
-    public void testCallToFunctionWithSubjectResultOfAnotherFunctionCall() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("attr", "   My Value   ");
-        verifyEquals("${attr:trim():substring(2, 5)}", attributes, " Va");
-    }
-
-    @Test
-    public void testProblematic1() {
-        // There was a bug that prevented this expression from compiling. This test just verifies that it now compiles.
-        final String queryString = "${xx:append( \"120101\" ):toDate( 'yyMMddHHmmss' ):format( \"yy-MM-dd’T’HH:mm:ss\") }";
-        Query.compile(queryString);
-    }
-
-    @Test
-    public void testEquals() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("attr", " XX    ");
-        verifyEquals("${attr:trim():equals('XX')}", attributes, true);
-    }
-    
-    @Test
-    public void testDeeplyEmbedded() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("x", "false");
-        attributes.put("abc", "a");
-        attributes.put("a", "a");
-        
-        verifyEquals("${x:or( ${${abc}:length():equals(1)} )}", attributes, true);
-    }
-    
-    
-    @Test
-    public void testExtractExpressionRanges() {
-        List<Range> ranges = Query.extractExpressionRanges("hello");
-        assertTrue(ranges.isEmpty());
-        
-        ranges = Query.extractExpressionRanges("${hello");
-        assertTrue(ranges.isEmpty());
-        
-        ranges = Query.extractExpressionRanges("hello}");
-        assertTrue(ranges.isEmpty());
-        
-        ranges = Query.extractExpressionRanges("$${hello");
-        assertTrue(ranges.isEmpty());
-
-        ranges = Query.extractExpressionRanges("$he{ll}o");
-        assertTrue(ranges.isEmpty());
-
-        ranges = Query.extractExpressionRanges("${hello}");
-        assertEquals(1, ranges.size());
-        Range range = ranges.get(0);
-        assertEquals(0, range.getStart());
-        assertEquals(7, range.getEnd());
-        
-        ranges = Query.extractExpressionRanges("${hello:equals( ${goodbye} )}");
-        assertEquals(1, ranges.size());
-        range = ranges.get(0);
-        assertEquals(0, range.getStart());
-        assertEquals(28, range.getEnd());
-        
-        ranges = Query.extractExpressionRanges("${hello:equals( $${goodbye} )}");
-        assertEquals(1, ranges.size());
-        range = ranges.get(0);
-        assertEquals(0, range.getStart());
-        assertEquals(29, range.getEnd());
-        
-        ranges = Query.extractExpressionRanges("${hello:equals( $${goodbye} )} or just hi, ${bob:or(${jerry})}");
-        assertEquals(2, ranges.size());
-        range = ranges.get(0);
-        assertEquals(0, range.getStart());
-        assertEquals(29, range.getEnd());
-        
-        range = ranges.get(1);
-        assertEquals(43, range.getStart());
-        assertEquals(61, range.getEnd());
-        
-        
-        ranges = Query.extractExpressionRanges("${hello:equals( ${goodbye} )} or just hi, ${bob}, are you ${bob.age:toNumber()} yet? $$$${bob}");
-        assertEquals(3, ranges.size());
-        range = ranges.get(0);
-        assertEquals(0, range.getStart());
-        assertEquals(28, range.getEnd());
-        
-        range = ranges.get(1);
-        assertEquals(42, range.getStart());
-        assertEquals(47, range.getEnd());
-        
-        range = ranges.get(2);
-        assertEquals(58, range.getStart());
-        assertEquals(78, range.getEnd());
-        
-        ranges = Query.extractExpressionRanges("${x:matches( '.{4}' )}");
-        assertEquals(1, ranges.size());
-        range = ranges.get(0);
-        assertEquals(0, range.getStart());
-        assertEquals(21, range.getEnd());
-    }
-    
-    
-    @Test
-    public void testExtractExpressionTypes() {
-        List<ResultType> types = Query.extractResultTypes("${hello:equals( ${goodbye} )} or just hi, ${bob}, are you ${bob.age:toNumber()} yet? $$$${bob}");
-        assertEquals(3, types.size());
-        assertEquals(ResultType.BOOLEAN, types.get(0));
-        assertEquals(ResultType.STRING, types.get(1));
-        assertEquals(ResultType.NUMBER, types.get(2));
-    }
-    
-    
-    @Test
-    public void testEqualsEmbedded() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("x", "hello");
-        attributes.put("y", "good-bye");
-        
-        verifyEquals("${x:equals( ${y} )}", attributes, false);
-        
-        attributes.put("y", "hello");
-        verifyEquals("${x:equals( ${y} )}", attributes, true);
-        
-        attributes.put("x", "4");
-        attributes.put("y", "3");
-        attributes.put("z", "1");
-        attributes.put("h", "100");
-        verifyEquals("${x:toNumber():lt( ${y:toNumber():plus( ${h:toNumber()} )} )}", attributes, true);
-        verifyEquals("${h:toNumber():ge( ${y:toNumber():plus( ${z:toNumber()} )} )}", attributes, true);
-        verifyEquals("${x:toNumber():equals( ${y:toNumber():plus( ${z:toNumber()} )} )}", attributes, true);
-
-        attributes.put("x", "88");
-        verifyEquals("${x:toNumber():gt( ${y:toNumber():plus( ${z:toNumber()} )} )}", attributes, true);
-
-        attributes.put("y", "88");
-        assertEquals("true", Query.evaluateExpressions("${x:equals( '${y}' )}", attributes, null));
-    }
-    
-    
-    @Test
-    public void testComplicatedEmbeddedExpressions() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("fox", "quick, brown");
-        attributes.put("dog", "lazy");
-        
-        verifyEquals("${fox:substring( ${ 'dog' :substring(2):length()}, 5 ):equals( 'ick' )}", attributes, true);
-        verifyEquals("${fox:substring( ${ 'dog' :substring(2):length()}, 5 ):equals( 'ick' )}", attributes, true);
-    }
-    
-    @Test
-    public void testQuotingQuotes() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("xx", "say 'hi'");
-
-        String query = "${xx:replaceAll( \"'.*'\", '\\\"hello\\\"' )}";
-        verifyEquals(query, attributes, "say \"hello\"");
-
-        query = "${xx:replace( \"'\", '\"')}";
-        verifyEquals(query, attributes, "say \"hi\"");
-
-        query = "${xx:replace( '\\'', '\"')}";
-        System.out.println(query);
-        verifyEquals(query, attributes, "say \"hi\"");
-    }
-    
-    @Test
-    public void testDoubleQuotesWithinSingleQuotes() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("xx", "say 'hi'");
-
-        String query = "${xx:replace( \"'hi'\", '\\\"hello\\\"' )}";
-        System.out.println(query);
-        verifyEquals(query, attributes, "say \"hello\"");
-    }
-    
-    @Test
-    public void testEscapeQuotes() {
-        final long timestamp = 1403620278642L;
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("date", String.valueOf(timestamp));
-        
-        final String format = "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'";
-        
-        final String query = "startDateTime=\"${date:toNumber():toDate():format(\"" + format + "\")}\"";
-        final String result = Query.evaluateExpressions(query, attributes, null);
-        
-        final String expectedTime = new SimpleDateFormat(format, Locale.US).format(timestamp);
-        assertEquals("startDateTime=\"" + expectedTime + "\"", result);
-        
-        final List<Range> ranges = Query.extractExpressionRanges(query);
-        assertEquals(1, ranges.size());
-    }
-    
-    @Test
-    public void testDateConversion() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("date", "1403620278642");
-        
-        verifyEquals("${date:format('yyyy')}", attributes, "2014");
-        verifyEquals("${date:toDate():format('yyyy')}", attributes, "2014");
-        verifyEquals("${date:toNumber():format('yyyy')}", attributes, "2014");
-        verifyEquals("${date:toNumber():toDate():format('yyyy')}", attributes, "2014");
-        verifyEquals("${date:toDate():toNumber():format('yyyy')}", attributes, "2014");
-        verifyEquals("${date:toDate():toNumber():toDate():toNumber():toDate():toNumber():format('yyyy')}", attributes, "2014");
-    }
-    
-    @Test
-    public void testSingleLetterAttribute() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("A", "0123456789");
-        
-        verifyEquals("${A}", attributes, "0123456789");
-        verifyEquals("${'A'}", attributes, "0123456789");
-    }
-
-    
-    @Test
-    public void testImplicitConversions() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("A", "0123456789");
-        attributes.put("b", "true");
-        attributes.put("c", "false");
-        attributes.put("d", "Quick Brown Fox");
-        attributes.put("F", "-48");
-        attributes.put("n", "2014/04/04 00:00:00");
-        
-        final Calendar cal = Calendar.getInstance();
-        cal.set(Calendar.YEAR, 2014);
-        cal.set(Calendar.MONTH, 3);
-        cal.set(Calendar.DAY_OF_MONTH, 4);
-        cal.set(Calendar.HOUR, 0);
-        cal.set(Calendar.MINUTE, 0);
-        cal.set(Calendar.SECOND, 45);
-        
-        final String dateString = cal.getTime().toString();
-        attributes.put("z", dateString);
-
-        
-        verifyEquals("${A:plus(4)}", attributes, 123456793L);
-        verifyEquals("${A:plus( ${F} )}", attributes, 123456741L);
-
-        verifyEquals("${F:lt( ${A} )}", attributes, true);
-        verifyEquals("${A:substring(2,3):plus(21):substring(1,2):plus(0)}", attributes, 3L);
-        verifyEquals("${n:format( 'yyyy' )}", attributes, "2014");
-        verifyEquals("${z:format( 'yyyy' )}", attributes, "2014");
-        
-        attributes.put("n", "2014/04/04 00:00:00.045");
-        verifyEquals("${n:format( 'yyyy' ):append(','):append( ${n:format( 'SSS' )} )}", attributes, "2014,045");
-    }
-    
-    @Test
-    public void testNewLinesAndTabsInQuery() {
-        final String query = "${ abc:equals('abc'):or( \n\t${xx:isNull()}\n) }";
-        assertEquals(ResultType.BOOLEAN, Query.getResultType(query));
-        Query.validateExpression(query, false);
-        assertEquals("true", Query.evaluateExpressions(query));
-    }
-    
-    @Test
-    public void testAttributeReferencesWithWhiteSpace() {
-        final Map<String, String> attrs = new HashMap<>();
-        attrs.put("a b c,d", "abc");
-        
-        final String query = "${ 'a b c,d':equals('abc') }";
-        verifyEquals(query, attrs, true);
-    }
-
-    @Test
-    public void testComments() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("abc", "xyz");
-
-        final String expression = 
-            "# hello, world\n" + 
-            "${# ref attr\n" +
-            "\t" +
-                "abc" +
-            "\t" +
-                "#end ref attr\n" +
-            "}";
-
-        Query query = Query.compile(expression);
-        QueryResult<?> result = query.evaluate(attributes);
-        assertEquals(ResultType.STRING, result.getResultType());
-        assertEquals("xyz", result.getValue());
-        
-        query = Query.compile("${abc:append('# hello') #good-bye \n}");
-        result = query.evaluate(attributes);
-        assertEquals(ResultType.STRING, result.getResultType());
-        assertEquals("xyz# hello", result.getValue());
-    }
-    
-    @Test
-    public void testAppendPrepend() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("attr", "XX");
-        attributes.put("YXXX", "bingo");
-        
-        verifyEquals("${${attr:append('X'):prepend('Y')}}", attributes, "bingo");
-    }
-    
-    @Test
-    public void testIsNull() {
-        final Map<String, String> attributes = new HashMap<>();
-        verifyEquals("${attr:isNull()}", attributes, true);
-    }
-    
-    @Test
-    public void testNotNull() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("attr", "");
-        
-        verifyEquals("${attr:notNull()}", attributes, true);
-    }
-    
-    @Test
-    public void testIsNullOrLengthEquals0() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("abc", "");
-        attributes.put("xyz", "xyz");
-        attributes.put("xx", "  ");
-        
-        verifyEquals("${abc:isNull():or( ${abc:length():equals(0)} )}", attributes, true);
-        verifyEquals("${xyz:isNull():or( ${xyz:length():equals(0)} )}", attributes, false);
-        verifyEquals("${none:isNull():or( ${none:length():equals(0)} )}", attributes, true);
-        verifyEquals("${xx:isNull():or( ${xx:trim():length():equals(0)} )}", attributes, true);
-    }
-    
-    @Test
-    public void testReplaceNull() {
-        final Map<String, String> attributes = new HashMap<>();
-        verifyEquals("${attr:replaceNull('hello')}", attributes, "hello");
-    }
-    
-    @Test
-    public void testReplace() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("attr", "hello");
-        verifyEquals("${attr:replace('hell', 'yell')}", attributes, "yello");
-    }
-
-    @Test
-    public void testReplaceAll() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("attr", "hello");
-        attributes.put("xyz", "00-00TEST.2014_01_01_000000_value");
-        
-        verifyEquals("${xyz:replaceAll(\"^([^.]+)\\.([0-9]{4})_([0-9]{2})_([0-9]{2}).*$\", \"$3\")}", attributes, "01");
-        verifyEquals("${attr:replaceAll('l+', 'r')}", attributes, "hero");
-        
-        attributes.clear();
-        attributes.put("filename1", "abc.gz");
-        attributes.put("filename2", "abc.g");
-        attributes.put("filename3", "abc.gz.gz");
-        attributes.put("filename4", "abc.gz.g");
-        attributes.put("abc", "hello world");
-
-        verifyEquals("${filename3:replaceAll('\\\\\\.gz$', '')}", attributes, "abc.gz.gz");
-        verifyEquals("${filename3:replaceAll('\\\\\\\\.gz$', '')}", attributes, "abc.gz.gz");
-        verifyEquals("${filename1:replaceAll('\\.gz$', '')}", attributes, "abc");
-        verifyEquals("${filename2:replaceAll('\\.gz$', '')}", attributes, "abc.g");
-        verifyEquals("${filename4:replaceAll('\\\\.gz$', '')}", attributes, "abc.gz.g");
-
-        verifyEquals("${abc:replaceAll( 'lo wor(ld)', '$0')}", attributes, "hello world");
-        verifyEquals("${abc:replaceAll( 'he(llo) world', '$1')}", attributes, "llo");
-        verifyEquals("${abc:replaceAll( 'xx', '$0')}", attributes, "hello world");
-        verifyEquals("${abc:replaceAll( '(xx)', '$1')}", attributes, "hello world");
-        verifyEquals("${abc:replaceAll( 'lo wor(ld)', '$1')}", attributes, "helld");
-        
-    }
-    
-    
-    @Test
-    public void testReplaceAllWithOddNumberOfBackslashPairs() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("filename", "C:\\temp\\.txt");
-
-        verifyEquals("${filename:replace('\\\\', '/')}", attributes, "C:/temp/.txt");
-        verifyEquals("${filename:replaceAll('\\\\\\\\', '/')}", attributes, "C:/temp/.txt");
-        verifyEquals("${filename:replaceAll('\\\\\\.txt$', '')}", attributes, "C:\\temp");
-    }
-    
-    @Test
-    public void testReplaceAllWithMatchingGroup() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("attr", "hello");
-        
-        verifyEquals("${attr:replaceAll('.*?(l+).*', '$1')}", attributes, "ll");
-    }
-    
-    @Test
-    public void testMathOperations() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("one", "1");
-        attributes.put("two", "2");
-        attributes.put("three", "3");
-        attributes.put("four", "4");
-        attributes.put("five", "5");
-        attributes.put("hundred", "100");
-
-        verifyEquals("${hundred:toNumber():multiply(2):divide(3):plus(1):mod(5)}", attributes, 2L);
-    }
-
-    @Test
-    public void testIndexOf() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("attr", "https://abc.go");
-        verifyEquals("${attr:indexOf('/')}", attributes, 6L);
-    }
-    
-    @Test
-    public void testDate() {
-        final Calendar now = Calendar.getInstance();
-        final int year = now.get(Calendar.YEAR);
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("entryDate", String.valueOf(now.getTimeInMillis()));
-        
-        verifyEquals("${entryDate:toNumber():toDate():format('yyyy')}", attributes, String.valueOf(year));
-        
-        attributes.clear();
-        attributes.put("month", "3");
-        attributes.put("day", "4");
-        attributes.put("year", "2013");
-        assertEquals("63", Query.evaluateExpressions("${year:append('/'):append(${month}):append('/'):append(${day}):toDate('yyyy/MM/dd'):format('D')}", attributes, null));
-        assertEquals("63", Query.evaluateExpressions("${year:append('/'):append('${month}'):append('/'):append('${day}'):toDate('yyyy/MM/dd'):format('D')}", attributes, null));
-
-        verifyEquals("${year:append('/'):append(${month}):append('/'):append(${day}):toDate('yyyy/MM/dd'):format('D')}", attributes, "63");
-    }
-    
-    @Test
-    public void testSystemProperty() {
-        System.setProperty("hello", "good-bye");
-        assertEquals("good-bye", Query.evaluateExpressions("${hello}"));
-        assertEquals("good-bye", Query.compile("${hello}").evaluate().getValue());
-    }
-    
-    @Test
-    public void testAnyAttribute() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("abc", "zzz");
-        attributes.put("xyz", "abc");
-        
-        verifyEquals("${anyAttribute('abc', 'xyz', 'missingAttr'):substring(1,2):equals('b')}", attributes, true);
-        verifyEquals("${anyAttribute('abc', 'xyz'):substring(1,2):equals('b')}", attributes, true);
-        verifyEquals("${anyAttribute('xyz', 'abc'):substring(1,2):equals('b')}", attributes, true);
-        verifyEquals("${anyAttribute('zz'):substring(1,2):equals('b')}", attributes, false);
-        verifyEquals("${anyAttribute('abc', 'zz'):isNull()}", attributes, true);
-    }
-    
-    @Test
-    public void testAnyMatchingAttribute() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("abc", "zzz");
-        attributes.put("xyz", "abc");
-        attributes.put("123.cba", "hello");
-        
-        verifyEquals("${anyMatchingAttribute('.{2}x', '.{2}z'):substring(1,2):equals('b')}", attributes, true);
-        verifyEquals("${anyMatchingAttribute('.*'):substring(1,2):equals('b')}", attributes, true);
-        verifyEquals("${anyMatchingAttribute('x{44}'):substring(1,2):equals('b')}", attributes, false);
-        verifyEquals("${anyMatchingAttribute('abc'):substring(1,2):equals('b')}", attributes, false);
-        verifyEquals("${anyMatchingAttribute('xyz'):substring(1,2):equals('b')}", attributes, true);
-        verifyEquals("${anyMatchingAttribute('xyz'):notNull()}", attributes, true);
-        verifyEquals("${anyMatchingAttribute('xyz'):isNull()}", attributes, false);
-        verifyEquals("${anyMatchingAttribute('xxxxxxxxx'):notNull()}", attributes, false);
-        verifyEquals("${anyMatchingAttribute('123\\.c.*'):matches('hello')}", attributes, true);
-        verifyEquals("${anyMatchingAttribute('123\\.c.*|a.c'):matches('zzz')}", attributes, true);
-    }
-    
-    
-    @Test
-    public void testAnyDelineatedValue() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("abc", "a,b,c");
-        attributes.put("xyz", "abc");
-        
-        final String query = "${anyDelineatedValue('${abc}', ','):equals('b')}";
-        assertEquals(ResultType.BOOLEAN, Query.getResultType(query));
-        
-        assertEquals("true", Query.evaluateExpressions(query, attributes, null));
-        assertEquals("true", Query.evaluateExpressions("${anyDelineatedValue('${abc}', ','):equals('a')}", attributes, null));
-        assertEquals("true", Query.evaluateExpressions("${anyDelineatedValue('${abc}', ','):equals('c')}", attributes, null));
-        assertEquals("false", Query.evaluateExpressions("${anyDelineatedValue('${abc}', ','):equals('d')}", attributes, null));
-        
-        verifyEquals("${anyDelineatedValue(${abc}, ','):equals('b')}", attributes, true);
-        verifyEquals("${anyDelineatedValue(${abc}, ','):equals('a')}", attributes, true);
-        verifyEquals("${anyDelineatedValue(${abc}, ','):equals('c')}", attributes, true);
-        verifyEquals("${anyDelineatedValue(${abc}, ','):equals('d')}", attributes, false);
-    }
-    
-    @Test
-    public void testAllDelineatedValues() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("abc", "a,b,c");
-        attributes.put("xyz", "abc");
-        
-        final String query = "${allDelineatedValues('${abc}', ','):matches('[abc]')}";
-        
-        assertEquals(ResultType.BOOLEAN, Query.getResultType(query));
-        assertEquals("true", Query.evaluateExpressions(query, attributes, null));
-        assertEquals("true", Query.evaluateExpressions(query, attributes, null));
-        assertEquals("false", Query.evaluateExpressions("${allDelineatedValues('${abc}', ','):matches('[abd]')}", attributes, null));
-        assertEquals("false", Query.evaluateExpressions("${allDelineatedValues('${abc}', ','):equals('a'):not()}", attributes, null));
-        
-        verifyEquals("${allDelineatedValues(${abc}, ','):matches('[abc]')}", attributes, true);
-        verifyEquals("${allDelineatedValues(${abc}, ','):matches('[abd]')}", attributes, false);
-        verifyEquals("${allDelineatedValues(${abc}, ','):equals('a'):not()}", attributes, false);
-    }
-    
-    
-    @Test
-    public void testAllAttributes() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("abc", "1234");
-        attributes.put("xyz", "4132");
-        attributes.put("hello", "world!");
-
-        verifyEquals("${allAttributes('abc', 'xyz'):matches('\\d+')}", attributes, true);
-        verifyEquals("${allAttributes('abc', 'xyz'):toNumber():lt(99999)}", attributes, true);
-        verifyEquals("${allAttributes('abc', 'hello'):length():gt(3)}", attributes, true);
-        verifyEquals("${allAttributes('abc', 'hello'):length():equals(4)}", attributes, false);
-        verifyEquals("${allAttributes('abc', 'xyz'):length():equals(4)}", attributes, true);
-        verifyEquals("${allAttributes('abc', 'xyz', 'other'):isNull()}", attributes, false);
-        
-        try {
-            Query.compile("${allAttributes('#ah'):equals('hello')");
-            Assert.fail("Was able to compile with allAttributes and an invalid attribute name");
-        } catch (final AttributeExpressionLanguageParsingException e) {
-            // expected behavior
-        }
-    }
-    
-    
-    @Test
-    public void testMathOperators() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("abc", "1234");
-        attributes.put("xyz", "4132");
-        attributes.put("hello", "world!");
-
-        verifyEquals("${xyz:toNumber():gt( ${abc:toNumber()} )}", attributes, true);
-    }
-    
-    @Test
-    public void testAllMatchingAttributes() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("abc", "1234");
-        attributes.put("xyz", "4132");
-        attributes.put("hello", "world!");
-        attributes.put("123.cba", "hell.o");
-
-        System.out.println( printTree(Query.compile("${allMatchingAttributes('(abc|xyz)'):matches('\\\\d+')}").getTree()) );
-        
-        verifyEquals("${'123.cba':matches('hell\\.o')}", attributes, true);
-        verifyEquals("${allMatchingAttributes('123\\.cba'):equals('hell.o')}", attributes, true);
-        verifyEquals("${allMatchingAttributes('(abc|xyz)'):matches('\\d+')}", attributes, true);
-        verifyEquals("${allMatchingAttributes('[ax].*'):toNumber():lt(99999)}", attributes, true);
-        verifyEquals("${allMatchingAttributes('hell.'):length():gt(3)}", attributes, true);
-        
-        verifyEquals("${allMatchingAttributes('123\\.cba'):equals('no')}", attributes, false);
-    }
-    
-    @Test
-    public void testMatches() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("abc", "1234xyz4321");
-        attributes.put("end", "xyz");
-        attributes.put("xyz", "4132");
-        attributes.put("hello", "world!");
-        attributes.put("dotted", "abc.xyz");
-
-        final String evaluated = Query.evaluateExpressions("${abc:matches('1234${end}4321')}", attributes, null);
-        assertEquals("true", evaluated);
-        
-        attributes.put("end", "888");
-        final String secondEvaluation = Query.evaluateExpressions("${abc:matches('1234${end}4321')}", attributes, null);
-        assertEquals("false", secondEvaluation);
-        
-        verifyEquals("${dotted:matches('abc\\.xyz')}", attributes, true);
-   }
-    
-    
-    @Test
-    public void testFind() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("abc", "1234xyz4321");
-        attributes.put("end", "xyz");
-        attributes.put("xyz", "4132");
-        attributes.put("hello", "world!");
-        attributes.put("dotted", "abc.xyz");
-
-        final String evaluated = Query.evaluateExpressions("${abc:find('1234${end}4321')}", attributes, null);
-        assertEquals("true", evaluated);
-        
-        attributes.put("end", "888");
-        final String secondEvaluation = Query.evaluateExpressions("${abc:find('${end}4321')}", attributes, null);
-        assertEquals("false", secondEvaluation);
-        
-        verifyEquals("${dotted:find('\\.')}", attributes, true);
-   }
-    
-    @Test
-    public void testSubstringAfter() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("filename", "file-255");
-        
-        verifyEquals("${filename:substringAfter('')}", attributes, "file-255");
-        verifyEquals("${filename:substringAfterLast('')}", attributes, "file-255");
-        verifyEquals("${filename:substringBefore('')}", attributes, "file-255");
-        verifyEquals("${filename:substringBeforeLast('')}", attributes, "file-255");
-        verifyEquals("${filename:substringBefore('file')}", attributes, "");
-        
-        attributes.put("uri", "sftp://some.uri");
-        verifyEquals("${uri:substringAfter('sftp')}", attributes, "://some.uri");
-    }
-    
-    @Test
-    public void testSubstringAfterLast() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("filename", "file-file-255");
-        
-        verifyEquals("${filename:substringAfterLast('file-')}", attributes, "255");
-        verifyEquals("${filename:substringAfterLast('5')}", attributes, "");
-        verifyEquals("${filename:substringAfterLast('x')}", attributes, "file-file-255");
-    }
-    
-    @Test
-    public void testSubstringBefore() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("something", "some {} or other");
-        
-        verifyEquals("${something:substringBefore('}')}", attributes, "some {");
-    }
-    
-    @Test
-    public void testSubstring() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("filename", "file-255");
-        
-        verifyEquals("${filename:substring(1, 2)}", attributes, "i");
-        verifyEquals("${filename:substring(4)}", attributes, "-255");
-    }
-    
-    @Test
-    public void testToRadix() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("filename", "file-255");
-        attributes.put("filename2", "file-99999");
-
-
-        verifyEquals("${filename:substringAfter('-'):toNumber():toRadix(16):toUpper()}", attributes, "FF");
-        verifyEquals("${filename:substringAfter('-'):toNumber():toRadix(16, 4):toUpper()}", attributes, "00FF");
-        verifyEquals("${filename:substringAfter('-'):toNumber():toRadix(36, 3):toUpper()}", attributes, "073");
-    }
-    
-    @Test
-    public void testDateFormatConversion() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("blue", "20130917162643");
-        verifyEquals("${blue:toDate('yyyyMMddHHmmss'):format(\"yyyy/MM/dd HH:mm:ss.SSS'Z'\")}", attributes, "2013/09/17 16:26:43.000Z");
-    }
-
-    
-    @Test
-    public void testNot() {
-        verifyEquals("${ab:notNull():not()}", new HashMap<String, String>(), true);
-    }
-    
-    @Test
-    public void testAttributesWithSpaces() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("ab", "abc");
-        attributes.put("a  b", "abc");
-        
-        verifyEquals("${ab}", attributes, "abc");
-        verifyEquals("${'a  b'}", attributes, "abc");
-        verifyEquals("${'a b':replaceNull('')}", attributes, "");
-    }
-    
-    @Test
-    public void testOr() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("filename1", "xabc");
-        attributes.put("filename2", "yabc");
-        attributes.put("filename3", "abcxy");
-
-        verifyEquals("${filename1:startsWith('x'):or(true)}", attributes, true);
-        verifyEquals("${filename1:startsWith('x'):or( ${filename1:startsWith('y')} )}", attributes, true);
-        verifyEquals("${filename2:startsWith('x'):or( ${filename2:startsWith('y')} )}", attributes, true);
-        verifyEquals("${filename3:startsWith('x'):or( ${filename3:startsWith('y')} )}", attributes, false);
-        verifyEquals("${filename1:startsWith('x'):or( ${filename2:startsWith('y')} )}", attributes, true);
-        verifyEquals("${filename2:startsWith('x'):or( ${filename1:startsWith('y')} )}", attributes, false);
-    }
-    
-    @Test
-    public void testAnd() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("filename1", "xabc");
-        attributes.put("filename2", "yabc");
-        attributes.put("filename 3", "abcxy");
-
-        verifyEquals("${filename1:startsWith('x'):and(true)}", attributes, true);
-        verifyEquals("${filename1:startsWith('x') : and( false )}", attributes, false);
-        verifyEquals("${filename1:startsWith('x'):and( ${filename1:startsWith('y')} )}", attributes, false);
-        verifyEquals("${filename2:startsWith('x'):and( ${filename2:startsWith('y')} )}", attributes, false);
-        verifyEquals("${filename3:startsWith('x'):and( ${filename3:startsWith('y')} )}", attributes, false);
-        verifyEquals("${filename1:startsWith('x'):and( ${filename2:startsWith('y')} )}", attributes, true);
-        verifyEquals("${filename2:startsWith('x'):and( ${filename1:startsWith('y')} )}", attributes, false);
-        verifyEquals("${filename1:startsWith('x'):and( ${'filename 3':endsWith('y')} )}", attributes, true);
-    }
-    
-    @Test
-    public void testAndOrNot() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("filename1", "xabc");
-        attributes.put("filename2", "yabc");
-        attributes.put("filename 3", "abcxy");
-
-        final String query = 
-            "${" +
-            "     'non-existing':notNull():not():and(" +                                        // true AND (
-            "     ${filename1:startsWith('y')" +                                                    // false
-            "     :or(" +                                                                           // or
-            "       ${ filename1:startsWith('x'):and(false) }" +                                    // false
-            "     ):or(" +                                                                          // or
-            "       ${ filename2:endsWith('xxxx'):or( ${'filename 3':length():gt(1)} ) }" +         // true )
-            "     )}" +
-            "     )" +
-            "}";
-        
-        System.out.println(query);
-        verifyEquals(query, attributes, true);
-    }
-    
-    @Test
-    public void testAndOrLogicWithAnyAll() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("filename1", "xabc");
-        attributes.put("filename2", "yabc");
-        attributes.put("filename 3", "abcxy");
-        
-        verifyEquals("${anyMatchingAttribute('filename.*'):contains('abc'):and( ${filename2:equals('yabc')} )}", attributes, true);
-        verifyEquals("${anyMatchingAttribute('filename.*'):contains('abc'):and( ${filename2:equals('xabc')} )}", attributes, false);
-        verifyEquals("${anyMatchingAttribute('filename.*'):contains('abc'):not():or( ${filename2:equals('yabc')} )}", attributes, true);
-        verifyEquals("${anyMatchingAttribute('filename.*'):contains('abc'):not():or( ${filename2:equals('xabc')} )}", attributes, false);
-    }
-    
-    @Test
-    public void testKeywords() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("UUID", "123");
-        verifyEquals("${ 'UUID':toNumber():equals(123) }", attributes, true);
-    }
-    
-    @Test
-    public void testEqualsNumber() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("abc", "123");
-        verifyEquals("${ abc:toNumber():equals(123) }", attributes, true);
-    }
-    
-    @Test
-    public void testSubjectAsEmbeddedExpressionWithSurroundChars() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("b", "x");
-        attributes.put("abcxcba", "hello");
-        
-        final String evaluated = Query.evaluateExpressions("${ 'abc${b}cba':substring(0, 1) }", attributes, null);
-        assertEquals("h", evaluated);
-    }
-    
-    @Test
-    public void testToNumberFunctionReturnsNumberType() {
-        assertEquals(ResultType.NUMBER, Query.getResultType("${header.size:toNumber()}"));
-    }
-    
-    @Test
-    public void testAnyAttributeEmbedded() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("a1", "test1");
-        attributes.put("b2", "2test");
-        attributes.put("c3", "3test3");
-        
-        final String query = "${a1:equals('test1'):and( ${anyAttribute('a1','b2','c3'):contains('2')})}";
-        verifyEquals(query, attributes, true);
-    }
-    
-    private void verifyEquals(final String expression, final Map<String, String> attributes, final Object expectedResult) {
-        Query.validateExpression(expression, false);
-        assertEquals(String.valueOf(expectedResult), Query.evaluateExpressions(expression, attributes, null));
-        
-        Query query = Query.compile(expression);
-        QueryResult<?> result = query.evaluate(attributes);
-        
-        if ( expectedResult instanceof Number ) {
-            assertEquals(ResultType.NUMBER, result.getResultType());
-        } else if ( expectedResult instanceof Boolean ) {
-            assertEquals(ResultType.BOOLEAN, result.getResultType());
-        } else {
-            assertEquals(ResultType.STRING, result.getResultType());
-        }
-        
-        assertEquals(expectedResult, result.getValue());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java b/commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
deleted file mode 100644
index 398a23b..0000000
--- a/commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
+++ /dev/null
@@ -1,92 +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.attribute.expression.language;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.junit.Ignore;
-import org.junit.Test;
-
-public class TestStandardPreparedQuery {
-
-    @Test
-    public void testSimpleReference() {
-        final Map<String, String> attrs = new HashMap<>();
-        attrs.put("xx", "world");
-        
-        assertEquals("world", evaluate("${xx}", attrs));
-        assertEquals("hello, world!", evaluate("hello, ${xx}!", attrs));
-    }
-    
-    @Test
-    public void testEmbeddedReference() {
-        final Map<String, String> attrs = new HashMap<>();
-        attrs.put("xx", "yy");
-        attrs.put("yy", "world");
-        
-        assertEquals("world", evaluate("${${xx}}", attrs));
-    }
-    
-    @Test
-    public void test10MIterations() {
-        final Map<String, String> attrs = new HashMap<>();
-        attrs.put("xx", "world");
-        
-        final StandardPreparedQuery prepared = (StandardPreparedQuery) Query.prepare("${xx}");
-        final long start = System.nanoTime();
-        for (int i=0; i < 10000000; i++) {
-            assertEquals( "world", prepared.evaluateExpressions(attrs, null) );
-        }
-        final long nanos = System.nanoTime() - start;
-        System.out.println(TimeUnit.NANOSECONDS.toMillis(nanos));
-    }
-    
-    @Test
-    @Ignore("Takes too long")
-    public void test10MIterationsWithQuery() {
-        final Map<String, String> attrs = new HashMap<>();
-        attrs.put("xx", "world");
-
-        final long start = System.nanoTime();
-        for (int i=0; i < 10000000; i++) {
-            assertEquals( "world", Query.evaluateExpressions("${xx}", attrs) );
-        }
-        final long nanos = System.nanoTime() - start;
-        System.out.println(TimeUnit.NANOSECONDS.toMillis(nanos));
-
-    }
-    
-    @Test
-    public void testSeveralSequentialExpressions() {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("audience", "World");
-        attributes.put("comma", ",");
-        attributes.put("question", " how are you?");
-        assertEquals("Hello, World, how are you?!", evaluate("Hello, ${audience}${comma}${question}!", attributes));
-
-    }
-    
-    private String evaluate(final String query, final Map<String, String> attrs) {
-        final String evaluated = ((StandardPreparedQuery) Query.prepare(query)).evaluateExpressions(attrs, null);
-        return evaluated;
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-logging-utils/pom.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-logging-utils/pom.xml b/commons/nifi-logging-utils/pom.xml
deleted file mode 100644
index a6c3f44..0000000
--- a/commons/nifi-logging-utils/pom.xml
+++ /dev/null
@@ -1,36 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-commons-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-
-    <artifactId>nifi-logging-utils</artifactId>
-    <version>0.0.1-SNAPSHOT</version>
-    <name>NiFi Logging Utils</name>
-    <description>Utilities for logging</description>
-    
-    <dependencies>
-        <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>slf4j-api</artifactId>
-        </dependency>
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-logging-utils/src/main/java/org/apache/nifi/logging/NiFiLog.java
----------------------------------------------------------------------
diff --git a/commons/nifi-logging-utils/src/main/java/org/apache/nifi/logging/NiFiLog.java b/commons/nifi-logging-utils/src/main/java/org/apache/nifi/logging/NiFiLog.java
deleted file mode 100644
index 7c71d85..0000000
--- a/commons/nifi-logging-utils/src/main/java/org/apache/nifi/logging/NiFiLog.java
+++ /dev/null
@@ -1,367 +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.logging;
-
-import org.slf4j.Logger;
-import org.slf4j.Marker;
-
-/**
- *
- * @author unattributed
- */
-public class NiFiLog implements Logger {
-
-    private final Logger logger;
-
-    public NiFiLog(final Logger logger) {
-        this.logger = logger;
-    }
-
-    public Logger getWrappedLog() {
-        return logger;
-    }
-
-    @Override
-    public void warn(Marker marker, String string, Throwable thrwbl) {
-        if (logger.isDebugEnabled()) {
-            logger.warn(marker, string, thrwbl);
-        } else {
-            logger.warn(marker, string);
-        }
-    }
-
-    @Override
-    public void warn(Marker marker, String string, Object[] os) {
-        logger.warn(marker, string, os);
-    }
-
-    @Override
-    public void warn(Marker marker, String string, Object o, Object o1) {
-        logger.warn(marker, string, o, o1);
-    }
-
-    @Override
-    public void warn(Marker marker, String string, Object o) {
-        logger.warn(marker, string, o);
-    }
-
-    @Override
-    public void warn(Marker marker, String string) {
-        logger.warn(marker, string);
-    }
-
-    @Override
-    public void warn(String string, Throwable thrwbl) {
-        if (logger.isDebugEnabled()) {
-            logger.warn(string, thrwbl);
-        } else {
-            logger.warn(string);
-        }
-    }
-
-    @Override
-    public void warn(String string, Object o, Object o1) {
-        logger.warn(string, o, o1);
-    }
-
-    @Override
-    public void warn(String string, Object[] os) {
-        logger.warn(string, os);
-    }
-
-    @Override
-    public void warn(String string, Object o) {
-        logger.warn(string, o);
-    }
-
-    @Override
-    public void warn(String string) {
-        logger.warn(string);
-    }
-
-    @Override
-    public void trace(Marker marker, String string, Throwable thrwbl) {
-        logger.trace(marker, string, thrwbl);
-    }
-
-    @Override
-    public void trace(Marker marker, String string, Object[] os) {
-        logger.trace(marker, string, os);
-    }
-
-    @Override
-    public void trace(Marker marker, String string, Object o, Object o1) {
-        logger.trace(marker, string, o, o1);
-    }
-
-    @Override
-    public void trace(Marker marker, String string, Object o) {
-        logger.trace(marker, string, o);
-    }
-
-    @Override
-    public void trace(Marker marker, String string) {
-        logger.trace(marker, string);
-    }
-
-    @Override
-    public void trace(String string, Throwable thrwbl) {
-        logger.trace(string, thrwbl);
-    }
-
-    @Override
-    public void trace(String string, Object[] os) {
-        logger.trace(string, os);
-    }
-
-    @Override
-    public void trace(String string, Object o, Object o1) {
-        logger.trace(string, o, o1);
-    }
-
-    @Override
-    public void trace(String string, Object o) {
-        logger.trace(string, o);
-    }
-
-    @Override
-    public void trace(String string) {
-        logger.trace(string);
-    }
-
-    @Override
-    public boolean isWarnEnabled(Marker marker) {
-        return logger.isWarnEnabled(marker);
-    }
-
-    @Override
-    public boolean isWarnEnabled() {
-        return logger.isWarnEnabled();
-    }
-
-    @Override
-    public boolean isTraceEnabled(Marker marker) {
-        return logger.isTraceEnabled(marker);
-    }
-
-    @Override
-    public boolean isTraceEnabled() {
-        return logger.isTraceEnabled();
-    }
-
-    @Override
-    public boolean isInfoEnabled(Marker marker) {
-        return logger.isInfoEnabled(marker);
-    }
-
-    @Override
-    public boolean isInfoEnabled() {
-        return logger.isInfoEnabled();
-    }
-
-    @Override
-    public boolean isErrorEnabled(Marker marker) {
-        return logger.isErrorEnabled(marker);
-    }
-
-    @Override
-    public boolean isErrorEnabled() {
-        return logger.isErrorEnabled();
-    }
-
-    @Override
-    public boolean isDebugEnabled(Marker marker) {
-        return logger.isDebugEnabled(marker);
-    }
-
-    @Override
-    public boolean isDebugEnabled() {
-        return logger.isDebugEnabled();
-    }
-
-    @Override
-    public void info(Marker marker, String string, Throwable thrwbl) {
-        if (logger.isDebugEnabled()) {
-            logger.info(marker, string, thrwbl);
-        } else {
-            logger.info(marker, string);
-        }
-    }
-
-    @Override
-    public void info(Marker marker, String string, Object[] os) {
-        logger.info(marker, string, os);
-    }
-
-    @Override
-    public void info(Marker marker, String string, Object o, Object o1) {
-        logger.info(marker, string, o, o1);
-    }
-
-    @Override
-    public void info(Marker marker, String string, Object o) {
-        logger.info(marker, string, o);
-    }
-
-    @Override
-    public void info(Marker marker, String string) {
-        logger.info(marker, string);
-    }
-
-    @Override
-    public void info(String string, Throwable thrwbl) {
-        if (logger.isDebugEnabled()) {
-            logger.info(string, thrwbl);
-        } else {
-            logger.info(string);
-        }
-    }
-
-    @Override
-    public void info(String string, Object[] os) {
-        logger.info(string, os);
-    }
-
-    @Override
-    public void info(String string, Object o, Object o1) {
-        logger.info(string, o, o1);
-    }
-
-    @Override
-    public void info(String string, Object o) {
-        logger.info(string, o);
-    }
-
-    @Override
-    public void info(String string) {
-        logger.info(string);
-    }
-
-    @Override
-    public String getName() {
-        return logger.getName();
-    }
-
-    @Override
-    public void error(Marker marker, String string, Throwable thrwbl) {
-        if (logger.isDebugEnabled()) {
-            logger.error(marker, string, thrwbl);
-        } else {
-            logger.error(marker, string);
-        }
-    }
-
-    @Override
-    public void error(Marker marker, String string, Object[] os) {
-        logger.error(marker, string, os);
-    }
-
-    @Override
-    public void error(Marker marker, String string, Object o, Object o1) {
-        logger.error(marker, string, o, o1);
-    }
-
-    @Override
-    public void error(Marker marker, String string, Object o) {
-        logger.error(marker, string, o);
-    }
-
-    @Override
-    public void error(Marker marker, String string) {
-        logger.error(marker, string);
-    }
-
-    @Override
-    public void error(String string, Throwable thrwbl) {
-        if (logger.isDebugEnabled()) {
-            logger.error(string, thrwbl);
-        } else {
-            logger.error(string);
-        }
-    }
-
-    @Override
-    public void error(String string, Object[] os) {
-        logger.error(string, os);
-    }
-
-    @Override
-    public void error(String string, Object o, Object o1) {
-        logger.error(string, o, o1);
-    }
-
-    @Override
-    public void error(String string, Object o) {
-        logger.error(string, o);
-    }
-
-    @Override
-    public void error(String string) {
-        logger.error(string);
-    }
-
-    @Override
-    public void debug(Marker marker, String string, Throwable thrwbl) {
-        logger.debug(marker, string, thrwbl);
-    }
-
-    @Override
-    public void debug(Marker marker, String string, Object[] os) {
-        logger.debug(marker, string, os);
-    }
-
-    @Override
-    public void debug(Marker marker, String string, Object o, Object o1) {
-        logger.debug(marker, string, o, o1);
-    }
-
-    @Override
-    public void debug(Marker marker, String string, Object o) {
-        logger.debug(marker, string, o);
-    }
-
-    @Override
-    public void debug(Marker marker, String string) {
-        logger.debug(marker, string);
-    }
-
-    @Override
-    public void debug(String string, Throwable thrwbl) {
-        logger.debug(string, thrwbl);
-    }
-
-    @Override
-    public void debug(String string, Object[] os) {
-        logger.debug(string, os);
-    }
-
-    @Override
-    public void debug(String string, Object o, Object o1) {
-        logger.debug(string, o, o1);
-    }
-
-    @Override
-    public void debug(String string, Object o) {
-        logger.debug(string, o);
-    }
-
-    @Override
-    public void debug(String string) {
-        logger.debug(string);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-properties/.gitignore
----------------------------------------------------------------------
diff --git a/commons/nifi-properties/.gitignore b/commons/nifi-properties/.gitignore
deleted file mode 100755
index 073c9fa..0000000
--- a/commons/nifi-properties/.gitignore
+++ /dev/null
@@ -1,3 +0,0 @@
-/target
-/target
-/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-properties/pom.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-properties/pom.xml b/commons/nifi-properties/pom.xml
deleted file mode 100644
index 43ef7c2..0000000
--- a/commons/nifi-properties/pom.xml
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-commons-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-
-    <artifactId>nifi-properties</artifactId>
-    <version>0.0.1-SNAPSHOT</version>
-    <name>NiFi Properties</name>
-
-</project>


[38/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LeakyBucketStreamThrottler.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LeakyBucketStreamThrottler.java b/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LeakyBucketStreamThrottler.java
deleted file mode 100644
index ae075b5..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LeakyBucketStreamThrottler.java
+++ /dev/null
@@ -1,331 +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.stream.io;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-public class LeakyBucketStreamThrottler implements StreamThrottler {
-
-    private final int maxBytesPerSecond;
-    private final BlockingQueue<Request> requestQueue = new LinkedBlockingQueue<Request>();
-    private final ScheduledExecutorService executorService;
-    private final AtomicBoolean shutdown = new AtomicBoolean(false);
-
-    public LeakyBucketStreamThrottler(final int maxBytesPerSecond) {
-        this.maxBytesPerSecond = maxBytesPerSecond;
-
-        executorService = Executors.newSingleThreadScheduledExecutor();
-        final Runnable task = new Drain();
-        executorService.scheduleAtFixedRate(task, 0, 1000, TimeUnit.MILLISECONDS);
-    }
-
-    @Override
-    public void close() {
-        this.shutdown.set(true);
-
-        executorService.shutdown();
-        try {
-            // Should not take more than 2 seconds because we run every second. If it takes more than
-            // 2 seconds, it is because the Runnable thread is blocking on a write; in this case,
-            // we will just ignore it and return
-            executorService.awaitTermination(2, TimeUnit.SECONDS);
-        } catch (InterruptedException e) {
-        }
-    }
-
-    @Override
-    public OutputStream newThrottledOutputStream(final OutputStream toWrap) {
-        return new OutputStream() {
-            @Override
-            public void write(final int b) throws IOException {
-                write(new byte[]{(byte) b}, 0, 1);
-            }
-
-            @Override
-            public void write(byte[] b) throws IOException {
-                write(b, 0, b.length);
-            }
-
-            @Override
-            public void write(byte[] b, int off, int len) throws IOException {
-                final InputStream in = new ByteArrayInputStream(b, off, len);
-                LeakyBucketStreamThrottler.this.copy(in, toWrap);
-            }
-
-            @Override
-            public void close() throws IOException {
-                toWrap.close();
-            }
-
-            @Override
-            public void flush() throws IOException {
-                toWrap.flush();
-            }
-        };
-    }
-
-    @Override
-    public InputStream newThrottledInputStream(final InputStream toWrap) {
-        return new InputStream() {
-            final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-
-            @Override
-            public int read() throws IOException {
-                final ByteArrayOutputStream baos = new ByteArrayOutputStream(1);
-                LeakyBucketStreamThrottler.this.copy(toWrap, baos, 1L);
-                if (baos.getBufferLength() < 1) {
-                    return -1;
-                }
-
-                return baos.getUnderlyingBuffer()[0] & 0xFF;
-            }
-
-            @Override
-            public int read(final byte[] b) throws IOException {
-                if(b.length == 0){
-                    return 0;
-                }
-                return read(b, 0, b.length);
-            }
-
-            @Override
-            public int read(byte[] b, int off, int len) throws IOException {
-                if ( len < 0 ) {
-                    throw new IllegalArgumentException();
-                }
-                if ( len == 0 ) {
-                    return 0;
-                }
-                
-                baos.reset();
-                final int copied = (int) LeakyBucketStreamThrottler.this.copy(toWrap, baos, len);
-                if (copied == 0) {
-                    return -1;
-                }
-                System.arraycopy(baos.getUnderlyingBuffer(), 0, b, off, copied);
-                return copied;
-            }
-
-            @Override
-            public void close() throws IOException {
-                toWrap.close();
-            }
-
-            @Override
-            public int available() throws IOException {
-                return toWrap.available();
-            }
-        };
-    }
-
-    @Override
-    public long copy(final InputStream in, final OutputStream out) throws IOException {
-        return copy(in, out, -1);
-    }
-
-    @Override
-    public long copy(final InputStream in, final OutputStream out, final long maxBytes) throws IOException {
-        long totalBytesCopied = 0;
-        boolean finished = false;
-        while (!finished) {
-            final long requestMax = (maxBytes < 0) ? Long.MAX_VALUE : maxBytes - totalBytesCopied;
-            final Request request = new Request(in, out, requestMax);
-            boolean transferred = false;
-            while (!transferred) {
-                if (shutdown.get()) {
-                    throw new IOException("Throttler shutdown");
-                }
-
-                try {
-                    transferred = requestQueue.offer(request, 1000, TimeUnit.MILLISECONDS);
-                } catch (final InterruptedException e) {
-                    throw new IOException("Interrupted", e);
-                }
-            }
-
-            final BlockingQueue<Response> responseQueue = request.getResponseQueue();
-            Response response = null;
-            while (response == null) {
-                try {
-                    if (shutdown.get()) {
-                        throw new IOException("Throttler shutdown");
-                    }
-                    response = responseQueue.poll(1000L, TimeUnit.MILLISECONDS);
-                } catch (InterruptedException e) {
-                    throw new IOException("Interrupted", e);
-                }
-            }
-
-            if (!response.isSuccess()) {
-                throw response.getError();
-            }
-
-            totalBytesCopied += response.getBytesCopied();
-            finished = (response.getBytesCopied() == 0) || (totalBytesCopied >= maxBytes && maxBytes > 0);
-        }
-
-        return totalBytesCopied;
-    }
-
-    /**
-     * This class is responsible for draining water from the leaky bucket. I.e.,
-     * it actually moves the data
-     */
-    private class Drain implements Runnable {
-
-        private final byte[] buffer;
-
-        public Drain() {
-            final int bufferSize = Math.min(4096, maxBytesPerSecond);
-            buffer = new byte[bufferSize];
-        }
-
-        @Override
-        public void run() {
-            final long start = System.currentTimeMillis();
-
-            int bytesTransferred = 0;
-            while (bytesTransferred < maxBytesPerSecond) {
-                final long maxMillisToWait = 1000 - (System.currentTimeMillis() - start);
-                if (maxMillisToWait < 1) {
-                    return;
-                }
-
-                try {
-                    final Request request = requestQueue.poll(maxMillisToWait, TimeUnit.MILLISECONDS);
-                    if (request == null) {
-                        return;
-                    }
-
-                    final BlockingQueue<Response> responseQueue = request.getResponseQueue();
-
-                    final OutputStream out = request.getOutputStream();
-                    final InputStream in = request.getInputStream();
-
-                    try {
-                        final long requestMax = request.getMaxBytesToCopy();
-                        long maxBytesToTransfer;
-                        if (requestMax < 0) {
-                            maxBytesToTransfer = Math.min(buffer.length, maxBytesPerSecond - bytesTransferred);
-                        } else {
-                            maxBytesToTransfer = Math.min(requestMax,
-                                    Math.min(buffer.length, maxBytesPerSecond - bytesTransferred));
-                        }
-                        maxBytesToTransfer = Math.max(1L, maxBytesToTransfer);
-
-                        final int bytesCopied = fillBuffer(in, maxBytesToTransfer);
-                        out.write(buffer, 0, bytesCopied);
-
-                        final Response response = new Response(true, bytesCopied);
-                        responseQueue.put(response);
-                        bytesTransferred += bytesCopied;
-                    } catch (final IOException e) {
-                        final Response response = new Response(e);
-                        responseQueue.put(response);
-                    }
-                } catch (InterruptedException e) {
-                }
-            }
-        }
-
-        private int fillBuffer(final InputStream in, final long maxBytes) throws IOException {
-            int bytesRead = 0;
-            int len;
-            while (bytesRead < maxBytes && (len = in.read(buffer, bytesRead, (int) Math.min(maxBytes - bytesRead, buffer.length - bytesRead))) > 0) {
-                bytesRead += len;
-            }
-
-            return bytesRead;
-        }
-    }
-
-    private static class Response {
-
-        private final boolean success;
-        private final IOException error;
-        private final int bytesCopied;
-
-        public Response(final boolean success, final int bytesCopied) {
-            this.success = success;
-            this.bytesCopied = bytesCopied;
-            this.error = null;
-        }
-
-        public Response(final IOException error) {
-            this.success = false;
-            this.error = error;
-            this.bytesCopied = -1;
-        }
-
-        public boolean isSuccess() {
-            return success;
-        }
-
-        public IOException getError() {
-            return error;
-        }
-
-        public int getBytesCopied() {
-            return bytesCopied;
-        }
-    }
-
-    private static class Request {
-
-        private final OutputStream out;
-        private final InputStream in;
-        private final long maxBytesToCopy;
-        private final BlockingQueue<Response> responseQueue;
-
-        public Request(final InputStream in, final OutputStream out, final long maxBytesToCopy) {
-            this.out = out;
-            this.in = in;
-            this.maxBytesToCopy = maxBytesToCopy;
-            this.responseQueue = new LinkedBlockingQueue<Response>(1);
-        }
-
-        public BlockingQueue<Response> getResponseQueue() {
-            return this.responseQueue;
-        }
-
-        public OutputStream getOutputStream() {
-            return out;
-        }
-
-        public InputStream getInputStream() {
-            return in;
-        }
-
-        public long getMaxBytesToCopy() {
-            return maxBytesToCopy;
-        }
-
-        @Override
-        public String toString() {
-            return "Request[maxBytes=" + maxBytesToCopy + "]";
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/NonCloseableInputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/NonCloseableInputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/NonCloseableInputStream.java
deleted file mode 100644
index 0e75a22..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/NonCloseableInputStream.java
+++ /dev/null
@@ -1,56 +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.stream.io;
-
-import java.io.FilterInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- * Wraps and InputStream so that the underlying InputStream cannot be closed.
- * This is used so that the InputStream can be wrapped with yet another
- * InputStream and prevent the outer layer from closing the inner InputStream
- */
-public class NonCloseableInputStream extends FilterInputStream {
-
-    private final InputStream toWrap;
-
-    public NonCloseableInputStream(final InputStream toWrap) {
-        super(toWrap);
-        this.toWrap = toWrap;
-    }
-
-    @Override
-    public int read() throws IOException {
-        return toWrap.read();
-    }
-
-    @Override
-    public int read(byte[] b) throws IOException {
-        return toWrap.read(b);
-    }
-
-    @Override
-    public int read(byte[] b, int off, int len) throws IOException {
-        return toWrap.read(b, off, len);
-    }
-
-    @Override
-    public void close() throws IOException {
-        // do nothing
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/NonCloseableOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/NonCloseableOutputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/NonCloseableOutputStream.java
deleted file mode 100644
index 9c77637..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/NonCloseableOutputStream.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.stream.io;
-
-import java.io.FilterOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-
-public class NonCloseableOutputStream extends FilterOutputStream {
-
-    private final OutputStream out;
-
-    public NonCloseableOutputStream(final OutputStream out) {
-        super(out);
-        this.out = out;
-    }
-
-    @Override
-    public void write(byte[] b) throws IOException {
-        out.write(b);
-    }
-
-    @Override
-    public void write(byte[] b, int off, int len) throws IOException {
-        out.write(b, off, len);
-    }
-
-    @Override
-    public void write(int b) throws IOException {
-        out.write(b);
-    }
-
-    @Override
-    public void close() throws IOException {
-        out.flush();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/NullOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/NullOutputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/NullOutputStream.java
deleted file mode 100644
index 8452761..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/NullOutputStream.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.stream.io;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-/**
- * OutputStream that throws away all data, just like as if writing to /dev/null
- */
-public class NullOutputStream extends OutputStream {
-
-    @Override
-    public void write(final int b) throws IOException {
-    }
-
-    @Override
-    public void write(final byte[] b) throws IOException {
-    }
-
-    @Override
-    public void write(final byte[] b, int off, int len) throws IOException {
-    }
-
-    @Override
-    public void close() throws IOException {
-    }
-
-    @Override
-    public void flush() throws IOException {
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/StreamThrottler.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/StreamThrottler.java b/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/StreamThrottler.java
deleted file mode 100644
index 9158050..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/StreamThrottler.java
+++ /dev/null
@@ -1,33 +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.stream.io;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-public interface StreamThrottler extends Closeable {
-
-    long copy(InputStream in, OutputStream out) throws IOException;
-
-    long copy(InputStream in, OutputStream out, long maxBytes) throws IOException;
-
-    InputStream newThrottledInputStream(final InputStream toWrap);
-
-    OutputStream newThrottledOutputStream(final OutputStream toWrap);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/StreamUtils.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/StreamUtils.java b/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/StreamUtils.java
deleted file mode 100644
index 8e3d606..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/StreamUtils.java
+++ /dev/null
@@ -1,257 +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.stream.io;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.nifi.stream.io.exception.BytePatternNotFoundException;
-import org.apache.nifi.stream.io.util.NonThreadSafeCircularBuffer;
-
-public class StreamUtils {
-
-    public static long copy(final InputStream source, final OutputStream destination) throws IOException {
-        final byte[] buffer = new byte[8192];
-        int len;
-        long totalCount = 0L;
-        while ((len = source.read(buffer)) > 0) {
-            destination.write(buffer, 0, len);
-            totalCount += len;
-        }
-        return totalCount;
-    }
-
-    /**
-     * Copies <code>numBytes</code> from <code>source</code> to
-     * <code>destination</code>. If <code>numBytes</code> are not available from
-     * <code>source</code>, throws EOFException
-     *
-     * @param source
-     * @param destination
-     * @param numBytes
-     * @throws IOException
-     */
-    public static void copy(final InputStream source, final OutputStream destination, final long numBytes) throws IOException {
-        final byte[] buffer = new byte[8192];
-        int len;
-        long bytesLeft = numBytes;
-        while ((len = source.read(buffer, 0, (int) Math.min(bytesLeft, buffer.length))) > 0) {
-            destination.write(buffer, 0, len);
-            bytesLeft -= len;
-        }
-
-        if (bytesLeft > 0) {
-            throw new EOFException("Attempted to copy " + numBytes + " bytes but only " + (numBytes - bytesLeft) + " bytes were available");
-        }
-    }
-
-    /**
-     * Reads data from the given input stream, copying it to the destination
-     * byte array. If the InputStream has less data than the given byte array,
-     * throws an EOFException
-     *
-     * @param source
-     * @param destination
-     * @throws IOException
-     */
-    public static void fillBuffer(final InputStream source, final byte[] destination) throws IOException {
-        fillBuffer(source, destination, true);
-    }
-
-    /**
-     * Reads data from the given input stream, copying it to the destination
-     * byte array. If the InputStream has less data than the given byte array,
-     * throws an EOFException if <code>ensureCapacity</code> is true and
-     * otherwise returns the number of bytes copied
-     *
-     * @param source
-     * @param destination
-     * @param ensureCapacity whether or not to enforce that the InputStream have
-     * at least as much data as the capacity of the destination byte array
-     * @return 
-     * @throws IOException
-     */
-    public static int fillBuffer(final InputStream source, final byte[] destination, final boolean ensureCapacity) throws IOException {
-        int bytesRead = 0;
-        int len;
-        while (bytesRead < destination.length) {
-            len = source.read(destination, bytesRead, destination.length - bytesRead);
-            if (len < 0) {
-                if (ensureCapacity) {
-                    throw new EOFException();
-                } else {
-                    break;
-                }
-            }
-
-            bytesRead += len;
-        }
-
-        return bytesRead;
-    }
-
-    /**
-     * Copies data from in to out until either we are out of data (returns null)
-     * or we hit one of the byte patterns identified by the
-     * <code>stoppers</code> parameter (returns the byte pattern matched). The
-     * bytes in the stopper will be copied.
-     *
-     * @param in
-     * @param out
-     * @param maxBytes
-     * @param stoppers
-     * @return the byte array matched, or null if end of stream was reached
-     * @throws IOException
-     */
-    public static byte[] copyInclusive(final InputStream in, final OutputStream out, final int maxBytes, final byte[]... stoppers) throws IOException {
-        if (stoppers.length == 0) {
-            return null;
-        }
-
-        final List<NonThreadSafeCircularBuffer> circularBuffers = new ArrayList<NonThreadSafeCircularBuffer>();
-        for (final byte[] stopper : stoppers) {
-            circularBuffers.add(new NonThreadSafeCircularBuffer(stopper));
-        }
-
-        long bytesRead = 0;
-        while (true) {
-            final int next = in.read();
-            if (next == -1) {
-                return null;
-            } else if (maxBytes > 0 && ++bytesRead >= maxBytes) {
-                throw new BytePatternNotFoundException("Did not encounter any byte pattern that was expected; data does not appear to be in the expected format");
-            }
-
-            out.write(next);
-
-            for (final NonThreadSafeCircularBuffer circ : circularBuffers) {
-                if (circ.addAndCompare((byte) next)) {
-                    return circ.getByteArray();
-                }
-            }
-        }
-    }
-
-    /**
-     * Copies data from in to out until either we are out of data (returns null)
-     * or we hit one of the byte patterns identified by the
-     * <code>stoppers</code> parameter (returns the byte pattern matched). The
-     * byte pattern matched will NOT be copied to the output and will be un-read
-     * from the input.
-     *
-     * @param in
-     * @param out
-     * @param maxBytes
-     * @param stoppers
-     * @return the byte array matched, or null if end of stream was reached
-     * @throws IOException
-     */
-    public static byte[] copyExclusive(final InputStream in, final OutputStream out, final int maxBytes, final byte[]... stoppers) throws IOException {
-        if (stoppers.length == 0) {
-            return null;
-        }
-
-        int longest = 0;
-        NonThreadSafeCircularBuffer longestBuffer = null;
-        final List<NonThreadSafeCircularBuffer> circularBuffers = new ArrayList<NonThreadSafeCircularBuffer>();
-        for (final byte[] stopper : stoppers) {
-            final NonThreadSafeCircularBuffer circularBuffer = new NonThreadSafeCircularBuffer(stopper);
-            if (stopper.length > longest) {
-                longest = stopper.length;
-                longestBuffer = circularBuffer;
-                circularBuffers.add(0, circularBuffer);
-            } else {
-                circularBuffers.add(circularBuffer);
-            }
-        }
-
-        long bytesRead = 0;
-        while (true) {
-            final int next = in.read();
-            if (next == -1) {
-                return null;
-            } else if (maxBytes > 0 && bytesRead++ > maxBytes) {
-                throw new BytePatternNotFoundException("Did not encounter any byte pattern that was expected; data does not appear to be in the expected format");
-            }
-
-            for (final NonThreadSafeCircularBuffer circ : circularBuffers) {
-                if (circ.addAndCompare((byte) next)) {
-                    // The longest buffer has some data that may not have been written out yet; we need to make sure
-                    // that we copy out those bytes.
-                    final int bytesToCopy = longest - circ.getByteArray().length;
-                    for (int i = 0; i < bytesToCopy; i++) {
-                        final int oldestByte = longestBuffer.getOldestByte();
-                        if (oldestByte != -1) {
-                            out.write(oldestByte);
-                            longestBuffer.addAndCompare((byte) 0);
-                        }
-                    }
-
-                    return circ.getByteArray();
-                }
-            }
-
-            if (longestBuffer.isFilled()) {
-                out.write(longestBuffer.getOldestByte());
-            }
-        }
-    }
-
-    /**
-     * Skips the specified number of bytes from the InputStream
-     *
-     * If unable to skip that number of bytes, throws EOFException
-     *
-     * @param stream
-     * @param bytesToSkip
-     * @throws IOException
-     */
-    public static void skip(final InputStream stream, final long bytesToSkip) throws IOException {
-        if (bytesToSkip <= 0) {
-            return;
-        }
-        long totalSkipped = 0L;
-
-        // If we have a FileInputStream, calling skip(1000000) will return 1000000 even if the file is only
-        // 3 bytes. As a result, we will skip 1 less than the number requested, and then read the last
-        // byte in order to make sure that we've consumed the number of bytes requested. We then check that
-        // the final byte, which we read, is not -1.
-        final long actualBytesToSkip = bytesToSkip - 1;
-        while (totalSkipped < actualBytesToSkip) {
-            final long skippedThisIteration = stream.skip(actualBytesToSkip - totalSkipped);
-            if (skippedThisIteration == 0) {
-                final int nextByte = stream.read();
-                if (nextByte == -1) {
-                    throw new EOFException();
-                } else {
-                    totalSkipped++;
-                }
-            }
-
-            totalSkipped += skippedThisIteration;
-        }
-
-        final int lastByte = stream.read();
-        if (lastByte == -1) {
-            throw new EOFException();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ZipOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ZipOutputStream.java b/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ZipOutputStream.java
deleted file mode 100644
index 2b9050d..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ZipOutputStream.java
+++ /dev/null
@@ -1,38 +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.stream.io;
-
-import java.io.OutputStream;
-
-/**
- * This class extends the {@link java.util.zip.ZipOutputStream} by providing a
- * constructor that allows the user to specify the compression level. The
- * default compression level is 1, as opposed to Java's default of 5.
- */
-public class ZipOutputStream extends java.util.zip.ZipOutputStream {
-
-    public static final int DEFAULT_COMPRESSION_LEVEL = 1;
-
-    public ZipOutputStream(final OutputStream out) {
-        this(out, DEFAULT_COMPRESSION_LEVEL);
-    }
-
-    public ZipOutputStream(final OutputStream out, final int compressionLevel) {
-        super(out);
-        def.setLevel(compressionLevel);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/exception/BytePatternNotFoundException.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/exception/BytePatternNotFoundException.java b/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/exception/BytePatternNotFoundException.java
deleted file mode 100644
index 5d08616..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/exception/BytePatternNotFoundException.java
+++ /dev/null
@@ -1,28 +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.stream.io.exception;
-
-import java.io.IOException;
-
-public class BytePatternNotFoundException extends IOException {
-
-    private static final long serialVersionUID = -4128911284318513973L;
-
-    public BytePatternNotFoundException(final String explanation) {
-        super(explanation);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/util/NonThreadSafeCircularBuffer.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/util/NonThreadSafeCircularBuffer.java b/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/util/NonThreadSafeCircularBuffer.java
deleted file mode 100644
index b4b4c17..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/util/NonThreadSafeCircularBuffer.java
+++ /dev/null
@@ -1,69 +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.stream.io.util;
-
-import java.util.Arrays;
-
-public class NonThreadSafeCircularBuffer {
-
-    private final byte[] lookingFor;
-    private final int[] buffer;
-    private int insertionPointer = 0;
-    private int bufferSize = 0;
-
-    public NonThreadSafeCircularBuffer(final byte[] lookingFor) {
-        this.lookingFor = lookingFor;
-        buffer = new int[lookingFor.length];
-        Arrays.fill(buffer, -1);
-    }
-
-    public byte[] getByteArray() {
-        return lookingFor;
-    }
-
-    /**
-     * Returns the oldest byte in the buffer
-     *
-     * @return
-     */
-    public int getOldestByte() {
-        return buffer[insertionPointer];
-    }
-
-    public boolean isFilled() {
-        return bufferSize >= buffer.length;
-    }
-
-    public boolean addAndCompare(final byte data) {
-        buffer[insertionPointer] = data;
-        insertionPointer = (insertionPointer + 1) % lookingFor.length;
-
-        bufferSize++;
-        if (bufferSize < lookingFor.length) {
-            return false;
-        }
-
-        for (int i = 0; i < lookingFor.length; i++) {
-            final byte compare = (byte) buffer[(insertionPointer + i) % lookingFor.length];
-            if (compare != lookingFor[i]) {
-                return false;
-            }
-        }
-
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java
deleted file mode 100644
index 92061e0..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java
+++ /dev/null
@@ -1,25 +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;
-
-public class BooleanHolder extends ObjectHolder<Boolean> {
-
-    public BooleanHolder(final boolean initialValue) {
-        super(initialValue);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/FormatUtils.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/FormatUtils.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/FormatUtils.java
deleted file mode 100644
index 805223f..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/FormatUtils.java
+++ /dev/null
@@ -1,205 +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.text.NumberFormat;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Locale;
-import java.util.concurrent.TimeUnit;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-public class FormatUtils {
-
-    private static final String UNION = "|";
-
-    // for Data Sizes
-    private static final double BYTES_IN_KILOBYTE = 1024;
-    private static final double BYTES_IN_MEGABYTE = BYTES_IN_KILOBYTE * 1024;
-    private static final double BYTES_IN_GIGABYTE = BYTES_IN_MEGABYTE * 1024;
-    private static final double BYTES_IN_TERABYTE = BYTES_IN_GIGABYTE * 1024;
-
-    // for Time Durations
-    private static final String NANOS = join(UNION, "ns", "nano", "nanos", "nanoseconds");
-    private static final String MILLIS = join(UNION, "ms", "milli", "millis", "milliseconds");
-    private static final String SECS = join(UNION, "s", "sec", "secs", "second", "seconds");
-    private static final String MINS = join(UNION, "m", "min", "mins", "minute", "minutes");
-    private static final String HOURS = join(UNION, "h", "hr", "hrs", "hour", "hours");
-    private static final String DAYS = join(UNION, "d", "day", "days");
-
-    private static final String VALID_TIME_UNITS = join(UNION, NANOS, MILLIS, SECS, MINS, HOURS, DAYS);
-    public static final String TIME_DURATION_REGEX = "(\\d+)\\s*(" + VALID_TIME_UNITS + ")";
-    public static final Pattern TIME_DURATION_PATTERN = Pattern.compile(TIME_DURATION_REGEX);
-
-    /**
-     * Formats the specified count by adding commas.
-     *
-     * @param count
-     * @return
-     */
-    public static String formatCount(final long count) {
-        return NumberFormat.getIntegerInstance().format(count);
-    }
-
-    /**
-     * Formats the specified duration in 'mm:ss.SSS' format.
-     *
-     * @param sourceDuration
-     * @param sourceUnit
-     * @return
-     */
-    public static String formatMinutesSeconds(final long sourceDuration, final TimeUnit sourceUnit) {
-        final long millis = TimeUnit.MILLISECONDS.convert(sourceDuration, sourceUnit);
-        final SimpleDateFormat formatter = new SimpleDateFormat("mm:ss.SSS", Locale.US);
-        return formatter.format(new Date(millis));
-    }
-
-    /**
-     * Formats the specified duration in 'HH:mm:ss.SSS' format.
-     *
-     * @param sourceDuration
-     * @param sourceUnit
-     * @return
-     */
-    public static String formatHoursMinutesSeconds(final long sourceDuration, final TimeUnit sourceUnit) {
-        final long millis = TimeUnit.MILLISECONDS.convert(sourceDuration, sourceUnit);
-        final long millisInHour = TimeUnit.MILLISECONDS.convert(1, TimeUnit.HOURS);
-        final int hours = (int) (millis / millisInHour);
-        final long whatsLeft = millis - hours * millisInHour;
-
-        return pad(hours) + ":" + new SimpleDateFormat("mm:ss.SSS", Locale.US).format(new Date(whatsLeft));
-    }
-
-    private static String pad(final int val) {
-        return (val < 10) ? "0" + val : String.valueOf(val);
-    }
-
-    /**
-     * Formats the specified data size in human readable format.
-     *
-     * @param dataSize Data size in bytes
-     * @return Human readable format
-     */
-    public static String formatDataSize(final double dataSize) {
-        // initialize the formatter
-        final NumberFormat format = NumberFormat.getNumberInstance();
-        format.setMaximumFractionDigits(2);
-
-        // check terabytes
-        double dataSizeToFormat = dataSize / BYTES_IN_TERABYTE;
-        if (dataSizeToFormat > 1) {
-            return format.format(dataSizeToFormat) + " TB";
-        }
-
-        // check gigabytes
-        dataSizeToFormat = dataSize / BYTES_IN_GIGABYTE;
-        if (dataSizeToFormat > 1) {
-            return format.format(dataSizeToFormat) + " GB";
-        }
-
-        // check megabytes
-        dataSizeToFormat = dataSize / BYTES_IN_MEGABYTE;
-        if (dataSizeToFormat > 1) {
-            return format.format(dataSizeToFormat) + " MB";
-        }
-
-        // check kilobytes
-        dataSizeToFormat = dataSize / BYTES_IN_KILOBYTE;
-        if (dataSizeToFormat > 1) {
-            return format.format(dataSizeToFormat) + " KB";
-        }
-
-        // default to bytes
-        return format.format(dataSize) + " bytes";
-    }
-
-    public static long getTimeDuration(final String value, final TimeUnit desiredUnit) {
-        final Matcher matcher = TIME_DURATION_PATTERN.matcher(value.toLowerCase());
-        if (!matcher.matches()) {
-            throw new IllegalArgumentException("Value '" + value + "' is not a valid Time Duration");
-        }
-
-        final String duration = matcher.group(1);
-        final String units = matcher.group(2);
-        TimeUnit specifiedTimeUnit = null;
-        switch (units.toLowerCase()) {
-            case "ns":
-            case "nano":
-            case "nanos":
-            case "nanoseconds":
-                specifiedTimeUnit = TimeUnit.NANOSECONDS;
-                break;
-            case "ms":
-            case "milli":
-            case "millis":
-            case "milliseconds":
-                specifiedTimeUnit = TimeUnit.MILLISECONDS;
-                break;
-            case "s":
-            case "sec":
-            case "secs":
-            case "second":
-            case "seconds":
-                specifiedTimeUnit = TimeUnit.SECONDS;
-                break;
-            case "m":
-            case "min":
-            case "mins":
-            case "minute":
-            case "minutes":
-                specifiedTimeUnit = TimeUnit.MINUTES;
-                break;
-            case "h":
-            case "hr":
-            case "hrs":
-            case "hour":
-            case "hours":
-                specifiedTimeUnit = TimeUnit.HOURS;
-                break;
-            case "d":
-            case "day":
-            case "days":
-                specifiedTimeUnit = TimeUnit.DAYS;
-                break;
-        }
-
-        final long durationVal = Long.parseLong(duration);
-        return desiredUnit.convert(durationVal, specifiedTimeUnit);
-    }
-
-    public static String formatUtilization(final double utilization) {
-        return utilization + "%";
-    }
-
-    private static String join(final String delimiter, final String... values) {
-        if (values.length == 0) {
-            return "";
-        } else if (values.length == 1) {
-            return values[0];
-        }
-
-        final StringBuilder sb = new StringBuilder();
-        sb.append(values[0]);
-        for (int i = 1; i < values.length; i++) {
-            sb.append(delimiter).append(values[i]);
-        }
-
-        return sb.toString();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java
deleted file mode 100644
index 213bbc0..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java
+++ /dev/null
@@ -1,54 +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;
-
-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);
-    }
-
-    public int getAndDecrement() {
-        return getAndAdd(-1);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java
deleted file mode 100644
index ef70ce8..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java
+++ /dev/null
@@ -1,60 +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;
-
-/**
- * Wraps a Long value so that it can be declared <code>final</code> and still be
- * accessed from which 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.
- */
-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/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/NaiveSearchRingBuffer.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/NaiveSearchRingBuffer.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/NaiveSearchRingBuffer.java
deleted file mode 100644
index 85bfd96..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/NaiveSearchRingBuffer.java
+++ /dev/null
@@ -1,135 +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.Arrays;
-
-/**
- * <p>
- * A RingBuffer that can be used to scan byte sequences for subsequences.
- * </p>
- *
- * <p>
- * This class implements an efficient naive search algorithm, which allows the
- * user of the library to identify byte sequences in a stream on-the-fly so that
- * the stream can be segmented without having to buffer the data.
- * </p>
- *
- * <p>
- * The intended usage paradigm is:
- * <code>
- * <pre>
- * final byte[] searchSequence = ...;
- * final CircularBuffer buffer = new CircularBuffer(searchSequence);
- * while ((int nextByte = in.read()) > 0) {
- *      if ( buffer.addAndCompare(nextByte) ) {
- *          // This byte is the last byte in the given sequence
- *      } else {
- *          // This byte does not complete the given sequence
- *      }
- * }
- * </pre>
- * </code>
- * </p>
- */
-public class NaiveSearchRingBuffer {
-
-    private final byte[] lookingFor;
-    private final int[] buffer;
-    private int insertionPointer = 0;
-    private int bufferSize = 0;
-
-    public NaiveSearchRingBuffer(final byte[] lookingFor) {
-        this.lookingFor = lookingFor;
-        this.buffer = new int[lookingFor.length];
-        Arrays.fill(buffer, -1);
-    }
-
-    /**
-     * Returns the contents of the internal buffer, which represents the last X
-     * bytes added to the buffer, where X is the minimum of the number of bytes
-     * added to the buffer or the length of the byte sequence for which we are
-     * looking
-     *
-     * @return
-     */
-    public byte[] getBufferContents() {
-        final int contentLength = Math.min(lookingFor.length, bufferSize);
-        final byte[] contents = new byte[contentLength];
-        for (int i = 0; i < contentLength; i++) {
-            final byte nextByte = (byte) buffer[(insertionPointer + i) % lookingFor.length];
-            contents[i] = nextByte;
-        }
-        return contents;
-    }
-
-    /**
-     * Returns the oldest byte in the buffer
-     *
-     * @return
-     */
-    public int getOldestByte() {
-        return buffer[insertionPointer];
-    }
-
-    /**
-     * Returns <code>true</code> if the number of bytes that have been added to
-     * the buffer is at least equal to the length of the byte sequence for which
-     * we are searching
-     *
-     * @return
-     */
-    public boolean isFilled() {
-        return bufferSize >= buffer.length;
-    }
-
-    /**
-     * Clears the internal buffer so that a new search may begin
-     */
-    public void clear() {
-        Arrays.fill(buffer, -1);
-        insertionPointer = 0;
-        bufferSize = 0;
-    }
-
-    /**
-     * Add the given byte to the buffer and notify whether or not the byte
-     * completes the desired byte sequence.
-     *
-     * @param data
-     * @return <code>true</code> if this byte completes the byte sequence,
-     * <code>false</code> otherwise.
-     */
-    public boolean addAndCompare(final byte data) {
-        buffer[insertionPointer] = data;
-        insertionPointer = (insertionPointer + 1) % lookingFor.length;
-
-        bufferSize++;
-        if (bufferSize < lookingFor.length) {
-            return false;
-        }
-
-        for (int i = 0; i < lookingFor.length; i++) {
-            final byte compare = (byte) buffer[(insertionPointer + i) % lookingFor.length];
-            if (compare != lookingFor[i]) {
-                return false;
-            }
-        }
-
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/ObjectHolder.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/ObjectHolder.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/ObjectHolder.java
deleted file mode 100644
index a58ec6a..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/ObjectHolder.java
+++ /dev/null
@@ -1,39 +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;
-
-/**
- * A bean that holds a single value of type T.
- *
- * @param <T>
- */
-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/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/RingBuffer.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/RingBuffer.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/RingBuffer.java
deleted file mode 100644
index c0bb830..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/RingBuffer.java
+++ /dev/null
@@ -1,292 +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.ArrayList;
-import java.util.List;
-import java.util.Objects;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-/**
- * Thread-safe implementation of a RingBuffer
- *
- * @param <T>
- */
-public class RingBuffer<T> {
-
-    private final Object[] buffer;
-    private int insertionPointer = 0;
-    private boolean filled = false;
-
-    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
-    private final Lock readLock = rwLock.readLock();
-    private final Lock writeLock = rwLock.writeLock();
-
-    public RingBuffer(final int size) {
-        buffer = new Object[size];
-    }
-
-    /**
-     * Adds the given value to the RingBuffer and returns the value that was
-     * removed in order to make room.
-     *
-     * @param value
-     * @return
-     */
-    @SuppressWarnings("unchecked")
-    public T add(final T value) {
-        Objects.requireNonNull(value);
-
-        writeLock.lock();
-        try {
-            final Object removed = buffer[insertionPointer];
-
-            buffer[insertionPointer] = value;
-
-            if (insertionPointer == buffer.length - 1) {
-                filled = true;
-            }
-
-            insertionPointer = (insertionPointer + 1) % buffer.length;
-            return (T) removed;
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    public int getSize() {
-        readLock.lock();
-        try {
-            return filled ? buffer.length : insertionPointer;
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    public List<T> getSelectedElements(final Filter<T> filter) {
-        return getSelectedElements(filter, Integer.MAX_VALUE);
-    }
-
-    public List<T> getSelectedElements(final Filter<T> filter, final int maxElements) {
-        final List<T> selected = new ArrayList<>(1000);
-        int numSelected = 0;
-        readLock.lock();
-        try {
-            for (int i = 0; i < buffer.length && numSelected < maxElements; i++) {
-                final int idx = (insertionPointer + i) % buffer.length;
-                final Object val = buffer[idx];
-                if (val == null) {
-                    continue;
-                }
-
-                @SuppressWarnings("unchecked")
-                final T element = (T) val;
-                if (filter.select(element)) {
-                    selected.add(element);
-                    numSelected++;
-                }
-            }
-        } finally {
-            readLock.unlock();
-        }
-        return selected;
-    }
-
-    public int countSelectedElements(final Filter<T> filter) {
-        int numSelected = 0;
-        readLock.lock();
-        try {
-            for (int i = 0; i < buffer.length; i++) {
-                final int idx = (insertionPointer + i) % buffer.length;
-                final Object val = buffer[idx];
-                if (val == null) {
-                    continue;
-                }
-
-                @SuppressWarnings("unchecked")
-                final T element = (T) val;
-                if (filter.select(element)) {
-                    numSelected++;
-                }
-            }
-        } finally {
-            readLock.unlock();
-        }
-
-        return numSelected;
-    }
-
-    /**
-     * Removes all elements from the RingBuffer that match the given filter
-     *
-     * @param filter
-     * @return
-     */
-    public int removeSelectedElements(final Filter<T> filter) {
-        int count = 0;
-
-        writeLock.lock();
-        try {
-            for (int i = 0; i < buffer.length; i++) {
-                final int idx = (insertionPointer + i + 1) % buffer.length;
-                final Object val = buffer[idx];
-                if (val == null) {
-                    continue;
-                }
-
-                @SuppressWarnings("unchecked")
-                final T element = (T) val;
-
-                if (filter.select(element)) {
-                    buffer[idx] = null;
-                }
-            }
-        } finally {
-            writeLock.unlock();
-        }
-
-        return count;
-    }
-
-    public List<T> asList() {
-        return getSelectedElements(new Filter<T>() {
-            @Override
-            public boolean select(final T value) {
-                return true;
-            }
-        });
-    }
-
-    public T getOldestElement() {
-        readLock.lock();
-        try {
-            return getElementData(insertionPointer);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    public T getNewestElement() {
-        readLock.lock();
-        try {
-            int index = (insertionPointer == 0) ? buffer.length : insertionPointer - 1;
-            return getElementData(index);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @SuppressWarnings("unchecked")
-    private T getElementData(final int index) {
-        readLock.lock();
-        try {
-            return (T) buffer[index];
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    /**
-     * Iterates over each element in the RingBuffer, calling the
-     * {@link ForEachEvaluator#evaluate(Object) evaluate} method on each element
-     * in the RingBuffer. If the Evaluator returns {@code false}, the method
-     * will skip all remaining elements in the RingBuffer; otherwise, the next
-     * element will be evaluated until all elements have been evaluated.
-     *
-     * @param evaluator
-     */
-    public void forEach(final ForEachEvaluator<T> evaluator) {
-        forEach(evaluator, IterationDirection.FORWARD);
-    }
-
-    /**
-     * Iterates over each element in the RingBuffer, calling the
-     * {@link ForEachEvaluator#evaluate(Object) evaluate} method on each element
-     * in the RingBuffer. If the Evaluator returns {@code false}, the method
-     * will skip all remaining elements in the RingBuffer; otherwise, the next
-     * element will be evaluated until all elements have been evaluated.
-     *
-     * @param evaluator
-     * @param iterationDirection the order in which to iterate over the elements
-     * in the RingBuffer
-     */
-    public void forEach(final ForEachEvaluator<T> evaluator, final IterationDirection iterationDirection) {
-        readLock.lock();
-        try {
-            final int startIndex;
-            final int endIndex;
-            final int increment;
-
-            if (iterationDirection == IterationDirection.FORWARD) {
-                startIndex = 0;
-                endIndex = buffer.length - 1;
-                increment = 1;
-            } else {
-                startIndex = buffer.length - 1;
-                endIndex = 0;
-                increment = -1;
-            }
-
-            for (int i = startIndex; (iterationDirection == IterationDirection.FORWARD ? i <= endIndex : i >= endIndex); i += increment) {
-                final int idx = (insertionPointer + i) % buffer.length;
-                final Object val = buffer[idx];
-                if (val == null) {
-                    continue;
-                }
-
-                @SuppressWarnings("unchecked")
-                final T element = (T) val;
-                if (!evaluator.evaluate(element)) {
-                    return;
-                }
-            }
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    public static interface Filter<S> {
-
-        boolean select(S value);
-    }
-
-    /**
-     * Defines an interface that can be used to iterate over all of the elements
-     * in the RingBuffer via the {@link #forEach} method
-     *
-     * @param <S>
-     */
-    public static interface ForEachEvaluator<S> {
-
-        /**
-         * Evaluates the given element and returns {@code true} if the next
-         * element should be evaluated, {@code false} otherwise
-         *
-         * @param value
-         * @return
-         */
-        boolean evaluate(S value);
-    }
-
-    public static enum IterationDirection {
-
-        FORWARD,
-        BACKWARD;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/StopWatch.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/StopWatch.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/StopWatch.java
deleted file mode 100644
index cd11930..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/StopWatch.java
+++ /dev/null
@@ -1,127 +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.TimeUnit;
-
-public final class StopWatch {
-
-    private long startNanos = -1L;
-    private long duration = -1L;
-
-    /**
-     * Creates a StopWatch but does not start it
-     */
-    public StopWatch() {
-        this(false);
-    }
-
-    /**
-     * @param autoStart whether or not the timer should be started automatically
-     */
-    public StopWatch(final boolean autoStart) {
-        if (autoStart) {
-            start();
-        }
-    }
-
-    public void start() {
-        this.startNanos = System.nanoTime();
-        this.duration = -1L;
-    }
-
-    public void stop() {
-        if (startNanos < 0) {
-            throw new IllegalStateException("StopWatch has not been started");
-        }
-        this.duration = System.nanoTime() - startNanos;
-        this.startNanos = -1L;
-    }
-
-    /**
-     * Returns the amount of time that the StopWatch was running.
-     *
-     * @param timeUnit
-     * @return
-     *
-     * @throws IllegalStateException if the StopWatch has not been stopped via
-     * {@link #stop()}
-     */
-    public long getDuration(final TimeUnit timeUnit) {
-        if (duration < 0) {
-            throw new IllegalStateException("Cannot get duration until StopWatch has been stopped");
-        }
-        return timeUnit.convert(duration, TimeUnit.NANOSECONDS);
-    }
-
-    /**
-     * Returns the amount of time that has elapsed since the timer was started.
-     *
-     * @param timeUnit
-     * @return
-     */
-    public long getElapsed(final TimeUnit timeUnit) {
-        return timeUnit.convert(System.nanoTime() - startNanos, TimeUnit.NANOSECONDS);
-    }
-
-    public String calculateDataRate(final long bytes) {
-        final double seconds = (double) duration / 1000000000.0D;
-        final long dataSize = (long) (bytes / seconds);
-        return FormatUtils.formatDataSize(dataSize) + "/sec";
-    }
-
-    public String getDuration() {
-        final StringBuilder sb = new StringBuilder();
-
-        long duration = this.duration;
-        final long minutes = (duration > 60000000000L) ? (duration / 60000000000L) : 0L;
-        duration -= TimeUnit.NANOSECONDS.convert(minutes, TimeUnit.MINUTES);
-
-        final long seconds = (duration > 1000000000L) ? (duration / 1000000000L) : 0L;
-        duration -= TimeUnit.NANOSECONDS.convert(seconds, TimeUnit.SECONDS);
-
-        final long millis = (duration > 1000000L) ? (duration / 1000000L) : 0L;
-        duration -= TimeUnit.NANOSECONDS.convert(millis, TimeUnit.MILLISECONDS);
-
-        final long nanos = duration % 1000000L;
-
-        if (minutes > 0) {
-            sb.append(minutes).append(" minutes");
-        }
-
-        if (seconds > 0) {
-            if (minutes > 0) {
-                sb.append(", ");
-            }
-
-            sb.append(seconds).append(" seconds");
-        }
-
-        if (millis > 0) {
-            if (seconds > 0) {
-                sb.append(", ");
-            }
-
-            sb.append(millis).append(" millis");
-        }
-        if (seconds == 0 && millis == 0) {
-            sb.append(nanos).append(" nanos");
-        }
-
-        return sb.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/Tuple.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/Tuple.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/Tuple.java
deleted file mode 100644
index 63736ed..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/Tuple.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.util;
-
-/**
- *
- * @author unattrib
- * @param <A>
- * @param <B>
- */
-public class Tuple<A, B> {
-
-    final A key;
-    final B value;
-
-    public Tuple(A key, B value) {
-        this.key = key;
-        this.value = value;
-    }
-
-    public A getKey() {
-        return key;
-    }
-
-    public B getValue() {
-        return value;
-    }
-
-    @Override
-    public boolean equals(final Object other) {
-        if (other == null) {
-            return false;
-        }
-        if (other == this) {
-            return true;
-        }
-        if (!(other instanceof Tuple)) {
-            return false;
-        }
-
-        final Tuple<?, ?> tuple = (Tuple<?, ?>) other;
-        if (key == null) {
-            if (tuple.key != null) {
-                return false;
-            }
-        } else {
-            if (!key.equals(tuple.key)) {
-                return false;
-            }
-        }
-
-        if (value == null) {
-            if (tuple.value != null) {
-                return false;
-            }
-        } else {
-            if (!value.equals(tuple.value)) {
-                return false;
-            }
-        }
-
-        return true;
-    }
-
-    @Override
-    public int hashCode() {
-        return 581 + (this.key == null ? 0 : this.key.hashCode()) + (this.value == null ? 0 : this.value.hashCode());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebugDisabledTimedLock.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebugDisabledTimedLock.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebugDisabledTimedLock.java
deleted file mode 100644
index a8d7e82..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebugDisabledTimedLock.java
+++ /dev/null
@@ -1,67 +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.concurrency;
-
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-
-public class DebugDisabledTimedLock implements DebuggableTimedLock {
-
-    private final Lock lock;
-
-    public DebugDisabledTimedLock(final Lock lock) {
-        this.lock = lock;
-    }
-
-    /**
-     *
-     * @return
-     */
-    @Override
-    public boolean tryLock() {
-        return lock.tryLock();
-    }
-
-    /**
-     *
-     * @param timeout
-     * @param timeUnit
-     * @return
-     */
-    @Override
-    public boolean tryLock(final long timeout, final TimeUnit timeUnit) {
-        try {
-            return lock.tryLock(timeout, timeUnit);
-        } catch (InterruptedException e) {
-            return false;
-        }
-    }
-
-    /**
-     *
-     */
-    @Override
-    public void lock() {
-        lock.lock();
-    }
-
-    @Override
-    public void unlock(final String task) {
-        lock.unlock();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebugEnabledTimedLock.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebugEnabledTimedLock.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebugEnabledTimedLock.java
deleted file mode 100644
index f082168..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebugEnabledTimedLock.java
+++ /dev/null
@@ -1,136 +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.concurrency;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class DebugEnabledTimedLock implements DebuggableTimedLock {
-
-    private final Lock lock;
-    private final Logger logger;
-    private long lockTime = 0L;
-
-    private final Map<String, Long> lockIterations = new HashMap<>();
-    private final Map<String, Long> lockNanos = new HashMap<>();
-
-    private final String name;
-    private final int iterationFrequency;
-
-    public DebugEnabledTimedLock(final Lock lock, final String name, final int iterationFrequency) {
-        this.lock = lock;
-        this.name = name;
-        this.iterationFrequency = iterationFrequency;
-        logger = LoggerFactory.getLogger(TimedLock.class.getName() + "." + name);
-    }
-
-    /**
-     *
-     * @return
-     */
-    @Override
-    public boolean tryLock() {
-        logger.trace("Trying to obtain Lock: {}", name);
-        final boolean success = lock.tryLock();
-        if (!success) {
-            logger.trace("TryLock failed for Lock: {}", name);
-            return false;
-        }
-        logger.trace("TryLock successful");
-
-        return true;
-    }
-
-    /**
-     *
-     * @param timeout
-     * @param timeUnit
-     * @return
-     */
-    @Override
-    public boolean tryLock(final long timeout, final TimeUnit timeUnit) {
-        logger.trace("Trying to obtain Lock {} with a timeout of {} {}", name, timeout, timeUnit);
-        final boolean success;
-        try {
-            success = lock.tryLock(timeout, timeUnit);
-        } catch (final InterruptedException ie) {
-            return false;
-        }
-
-        if (!success) {
-            logger.trace("TryLock failed for Lock {} with a timeout of {} {}", name, timeout, timeUnit);
-            return false;
-        }
-        logger.trace("TryLock successful");
-        return true;
-    }
-
-    /**
-     *
-     */
-    @Override
-    public void lock() {
-        logger.trace("Obtaining Lock {}", name);
-        lock.lock();
-        lockTime = System.nanoTime();
-        logger.trace("Obtained Lock {}", name);
-    }
-
-    /**
-     *
-     * @param task
-     */
-    @Override
-    public void unlock(final String task) {
-        if (lockTime <= 0L) {
-            lock.unlock();
-            return;
-        }
-
-        logger.trace("Releasing Lock {}", name);
-        final long nanosLocked = System.nanoTime() - lockTime;
-
-        Long startIterations = lockIterations.get(task);
-        if (startIterations == null) {
-            startIterations = 0L;
-        }
-        final long iterations = startIterations + 1L;
-        lockIterations.put(task, iterations);
-
-        Long startNanos = lockNanos.get(task);
-        if (startNanos == null) {
-            startNanos = 0L;
-        }
-        final long totalNanos = startNanos + nanosLocked;
-        lockNanos.put(task, totalNanos);
-
-        lockTime = -1L;
-
-        lock.unlock();
-        logger.trace("Released Lock {}", name);
-
-        if (iterations % iterationFrequency == 0) {
-            logger.debug("Lock {} held for {} nanos for task: {}; total lock iterations: {}; total lock nanos: {}", name, nanosLocked, task, iterations, totalNanos);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebuggableTimedLock.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebuggableTimedLock.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebuggableTimedLock.java
deleted file mode 100644
index 69da6e8..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebuggableTimedLock.java
+++ /dev/null
@@ -1,30 +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.concurrency;
-
-import java.util.concurrent.TimeUnit;
-
-public interface DebuggableTimedLock {
-
-    void lock();
-
-    boolean tryLock(long timePeriod, TimeUnit timeUnit);
-
-    boolean tryLock();
-
-    void unlock(String task);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/TimedLock.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/TimedLock.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/TimedLock.java
deleted file mode 100644
index 532d3c3..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/TimedLock.java
+++ /dev/null
@@ -1,59 +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.concurrency;
-
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TimedLock {
-
-    private final DebugEnabledTimedLock enabled;
-    private final DebugDisabledTimedLock disabled;
-
-    private final Logger logger;
-
-    public TimedLock(final Lock lock, final String name, final int iterationFrequency) {
-        this.enabled = new DebugEnabledTimedLock(lock, name, iterationFrequency);
-        this.disabled = new DebugDisabledTimedLock(lock);
-
-        logger = LoggerFactory.getLogger(TimedLock.class.getName() + "." + name);
-    }
-
-    private DebuggableTimedLock getLock() {
-        return logger.isDebugEnabled() ? enabled : disabled;
-    }
-
-    public boolean tryLock() {
-        return getLock().tryLock();
-    }
-
-    public boolean tryLock(final long timeout, final TimeUnit timeUnit) {
-        return getLock().tryLock(timeout, timeUnit);
-    }
-
-    public void lock() {
-        getLock().lock();
-    }
-
-    public void unlock(final String task) {
-        getLock().unlock(task);
-    }
-
-}


[07/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RootGroupPort.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RootGroupPort.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RootGroupPort.java
deleted file mode 100644
index 4afdfb7..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RootGroupPort.java
+++ /dev/null
@@ -1,78 +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.remote;
-
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.nifi.connectable.Port;
-import org.apache.nifi.remote.exception.BadRequestException;
-import org.apache.nifi.remote.exception.NotAuthorizedException;
-import org.apache.nifi.remote.exception.RequestExpiredException;
-import org.apache.nifi.remote.protocol.ServerProtocol;
-
-public interface RootGroupPort extends Port {
-
-    boolean isTransmitting();
-
-    void setGroupAccessControl(Set<String> groups);
-
-    Set<String> getGroupAccessControl();
-
-    void setUserAccessControl(Set<String> users);
-
-    Set<String> getUserAccessControl();
-
-    /**
-     * Verifies that the specified user is authorized to interact with this port
-     * and returns a {@link PortAuthorizationResult} indicating why the user is
-     * unauthorized if this assumption fails
-     *
-     * @param dn
-     * @return
-     */
-    PortAuthorizationResult checkUserAuthorization(String dn);
-
-    /**
-     * Receives data from the given stream
-     *
-     * @param peer
-     * @param serverProtocol
-     * @param requestHeaders
-     *
-     * @return the number of FlowFiles received
-     * @throws org.apache.nifi.remote.exception.NotAuthorizedException
-     * @throws org.apache.nifi.remote.exception.BadRequestException
-     * @throws org.apache.nifi.remote.exception.RequestExpiredException
-     */
-    int receiveFlowFiles(Peer peer, ServerProtocol serverProtocol, Map<String, String> requestHeaders) throws NotAuthorizedException, BadRequestException, RequestExpiredException;
-
-    /**
-     * Transfers data to the given stream
-     *
-     * @param peer
-     * @param requestHeaders
-     * @param serverProtocol
-     *
-     * @return the number of FlowFiles transferred
-     * @throws org.apache.nifi.remote.exception.NotAuthorizedException
-     * @throws org.apache.nifi.remote.exception.BadRequestException
-     * @throws org.apache.nifi.remote.exception.RequestExpiredException
-     */
-    int transferFlowFiles(Peer peer, ServerProtocol serverProtocol, Map<String, String> requestHeaders) throws NotAuthorizedException, BadRequestException, RequestExpiredException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/TransferDirection.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/TransferDirection.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/TransferDirection.java
deleted file mode 100644
index 56432d5..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/TransferDirection.java
+++ /dev/null
@@ -1,23 +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.remote;
-
-public enum TransferDirection {
-
-    SEND,
-    RECEIVE;
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java
deleted file mode 100644
index bfccd98..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java
+++ /dev/null
@@ -1,24 +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.remote;
-
-public interface VersionedRemoteResource {
-
-    VersionNegotiator getVersionNegotiator();
-
-    String getResourceName();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java
deleted file mode 100644
index b4206b3..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java
+++ /dev/null
@@ -1,79 +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.remote.codec;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.List;
-
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.remote.VersionedRemoteResource;
-import org.apache.nifi.remote.exception.ProtocolException;
-import org.apache.nifi.remote.exception.TransmissionDisabledException;
-
-/**
- * <p>
- * Provides a mechanism for encoding and decoding FlowFiles as streams so that
- * they can be transferred remotely.
- * </p>
- */
-public interface FlowFileCodec extends VersionedRemoteResource {
-
-    /**
-     * Returns a List of all versions that this codec is able to support, in the
-     * order that they are preferred by the codec
-     *
-     * @return
-     */
-    public List<Integer> getSupportedVersions();
-
-    /**
-     * Encodes a FlowFile and its content as a single stream of data and writes
-     * that stream to the output. If checksum is not null, it will be calculated
-     * as the stream is read
-     *
-     * @param flowFile the FlowFile to encode
-     * @param session a session that can be used to transactionally create and
-     * transfer flow files
-     * @param outStream the stream to write the data to
-     *
-     * @return the updated FlowFile
-     *
-     * @throws IOException
-     */
-    FlowFile encode(FlowFile flowFile, ProcessSession session, OutputStream outStream) throws IOException, TransmissionDisabledException;
-
-    /**
-     * Decodes the contents of the InputStream, interpreting the data to
-     * determine the next FlowFile's attributes and content, as well as their
-     * destinations. If not null, checksum will be used to calculate the
-     * checksum as the data is read.
-     *
-     * @param stream an InputStream containing FlowFiles' contents, attributes,
-     * and destinations
-     * @param session
-     *
-     * @return the FlowFile that was created, or <code>null</code> if the stream
-     * was out of data
-     *
-     * @throws IOException
-     * @throws ProtocolException if the input is malformed
-     */
-    FlowFile decode(InputStream stream, ProcessSession session) throws IOException, ProtocolException, TransmissionDisabledException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/BadRequestException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/BadRequestException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/BadRequestException.java
deleted file mode 100644
index f6c2f4f..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/BadRequestException.java
+++ /dev/null
@@ -1,30 +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.remote.exception;
-
-public class BadRequestException extends Exception {
-
-    private static final long serialVersionUID = -8034602852256106560L;
-
-    public BadRequestException(final String message) {
-        super(message);
-    }
-
-    public BadRequestException(final Throwable cause) {
-        super(cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java
deleted file mode 100644
index b61fc65..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java
+++ /dev/null
@@ -1,30 +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.remote.exception;
-
-public class HandshakeException extends Exception {
-
-    private static final long serialVersionUID = 178192341908726L;
-
-    public HandshakeException(final String message) {
-        super(message);
-    }
-
-    public HandshakeException(final Throwable cause) {
-        super(cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/NotAuthorizedException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/NotAuthorizedException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/NotAuthorizedException.java
deleted file mode 100644
index 24ff3a5..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/NotAuthorizedException.java
+++ /dev/null
@@ -1,26 +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.remote.exception;
-
-public class NotAuthorizedException extends Exception {
-
-    private static final long serialVersionUID = 2952623568114035498L;
-
-    public NotAuthorizedException(final String message) {
-        super(message);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java
deleted file mode 100644
index af0f467..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java
+++ /dev/null
@@ -1,26 +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.remote.exception;
-
-public class PortNotRunningException extends Exception {
-
-    private static final long serialVersionUID = -2790940982005516375L;
-
-    public PortNotRunningException(final String message) {
-        super(message);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
deleted file mode 100644
index 0f50b98..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
+++ /dev/null
@@ -1,34 +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.remote.exception;
-
-public class ProtocolException extends Exception {
-
-    private static final long serialVersionUID = 5763900324505818495L;
-
-    public ProtocolException(final String message, final Throwable cause) {
-        super(message, cause);
-    }
-
-    public ProtocolException(final String message) {
-        super(message);
-    }
-
-    public ProtocolException(final Throwable cause) {
-        super(cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/RequestExpiredException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/RequestExpiredException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/RequestExpiredException.java
deleted file mode 100644
index dd675b3..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/RequestExpiredException.java
+++ /dev/null
@@ -1,26 +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.remote.exception;
-
-/**
- * Used to indicate that by the time the request was serviced, it had already
- * expired
- */
-public class RequestExpiredException extends Exception {
-
-    private static final long serialVersionUID = -7037025330562827852L;
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java
deleted file mode 100644
index e6a0fe7..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java
+++ /dev/null
@@ -1,26 +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.remote.exception;
-
-public class UnknownPortException extends Exception {
-
-    private static final long serialVersionUID = -2790940982005516375L;
-
-    public UnknownPortException(final String message) {
-        super(message);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
deleted file mode 100644
index 32274eb..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
+++ /dev/null
@@ -1,78 +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.remote.protocol;
-
-import java.io.IOException;
-import java.util.Set;
-
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.remote.Peer;
-import org.apache.nifi.remote.PeerStatus;
-import org.apache.nifi.remote.VersionedRemoteResource;
-import org.apache.nifi.remote.codec.FlowFileCodec;
-import org.apache.nifi.remote.exception.HandshakeException;
-import org.apache.nifi.remote.exception.PortNotRunningException;
-import org.apache.nifi.remote.exception.ProtocolException;
-import org.apache.nifi.remote.exception.UnknownPortException;
-
-public interface ClientProtocol extends VersionedRemoteResource {
-
-    void handshake(Peer peer) throws IOException, HandshakeException, UnknownPortException, PortNotRunningException;
-
-    Set<PeerStatus> getPeerStatuses(Peer currentPeer) throws IOException, ProtocolException;
-
-    FlowFileCodec negotiateCodec(Peer peer) throws IOException, ProtocolException;
-
-    void receiveFlowFiles(Peer peer, ProcessContext context, ProcessSession session, FlowFileCodec codec) throws IOException, ProtocolException;
-
-    void transferFlowFiles(Peer peer, ProcessContext context, ProcessSession session, FlowFileCodec codec) throws IOException, ProtocolException;
-
-    void shutdown(Peer peer) throws IOException, ProtocolException;
-
-    boolean isReadyForFileTransfer();
-
-    /**
-     * returns <code>true</code> if remote instance indicates that the port is
-     * invalid
-     *
-     * @return
-     * @throws IllegalStateException if a handshake has not successfully
-     * completed
-     */
-    boolean isPortInvalid() throws IllegalStateException;
-
-    /**
-     * returns <code>true</code> if remote instance indicates that the port is
-     * unknown
-     *
-     * @return
-     * @throws IllegalStateException if a handshake has not successfully
-     * completed
-     */
-    boolean isPortUnknown();
-
-    /**
-     * returns <code>true</code> if remote instance indicates that the port's
-     * destination is full
-     *
-     * @return
-     * @throws IllegalStateException if a handshake has not successfully
-     * completed
-     */
-    boolean isDestinationFull();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java
deleted file mode 100644
index d2e2946..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java
+++ /dev/null
@@ -1,27 +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.remote.protocol;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-public interface CommunicationsInput {
-
-    InputStream getInputStream() throws IOException;
-
-    long getBytesRead();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java
deleted file mode 100644
index 95cab29..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java
+++ /dev/null
@@ -1,27 +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.remote.protocol;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-public interface CommunicationsOutput {
-
-    OutputStream getOutputStream() throws IOException;
-
-    long getBytesWritten();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java
deleted file mode 100644
index d009cec..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java
+++ /dev/null
@@ -1,64 +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.remote.protocol;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-public interface CommunicationsSession extends Closeable {
-
-    public static final byte[] MAGIC_BYTES = {(byte) 'N', (byte) 'i', (byte) 'F', (byte) 'i'};
-
-    CommunicationsInput getInput();
-
-    CommunicationsOutput getOutput();
-
-    void setTimeout(int millis) throws IOException;
-
-    int getTimeout() throws IOException;
-
-    void setUri(String uri);
-
-    String getUri();
-
-    String getUserDn();
-
-    void setUserDn(String dn);
-
-    boolean isDataAvailable();
-
-    long getBytesWritten();
-
-    long getBytesRead();
-
-    /**
-     * Asynchronously interrupts this FlowFileCodec. Implementations must ensure
-     * that they stop sending and receiving data as soon as possible after this
-     * method has been called, even if doing so results in sending only partial
-     * data to the peer. This will usually result in the peer throwing a
-     * SocketTimeoutException.
-     */
-    void interrupt();
-
-    /**
-     * Returns <code>true</code> if the connection is closed, <code>false</code>
-     * otherwise.
-     *
-     * @return
-     */
-    boolean isClosed();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/RequestType.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/RequestType.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/RequestType.java
deleted file mode 100644
index 41334fe..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/RequestType.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.remote.protocol;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-public enum RequestType {
-
-    NEGOTIATE_FLOWFILE_CODEC,
-    REQUEST_PEER_LIST,
-    SEND_FLOWFILES,
-    RECEIVE_FLOWFILES,
-    SHUTDOWN;
-
-    public void writeRequestType(final DataOutputStream dos) throws IOException {
-        dos.writeUTF(name());
-    }
-
-    public static RequestType readRequestType(final DataInputStream dis) throws IOException {
-        final String requestTypeVal = dis.readUTF();
-        try {
-            return RequestType.valueOf(requestTypeVal);
-        } catch (final Exception e) {
-            throw new IOException("Could not determine RequestType: received invalid value " + requestTypeVal);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java
deleted file mode 100644
index 0d18f2e..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java
+++ /dev/null
@@ -1,143 +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.remote.protocol;
-
-import java.io.IOException;
-
-import org.apache.nifi.cluster.NodeInformant;
-import org.apache.nifi.groups.ProcessGroup;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.remote.Peer;
-import org.apache.nifi.remote.RootGroupPort;
-import org.apache.nifi.remote.VersionedRemoteResource;
-import org.apache.nifi.remote.codec.FlowFileCodec;
-import org.apache.nifi.remote.exception.BadRequestException;
-import org.apache.nifi.remote.exception.HandshakeException;
-import org.apache.nifi.remote.exception.ProtocolException;
-
-public interface ServerProtocol extends VersionedRemoteResource {
-
-    /**
-     *
-     * @param rootGroup
-     */
-    void setRootProcessGroup(ProcessGroup rootGroup);
-
-    RootGroupPort getPort();
-
-    /**
-     * Optional operation. Sets the NodeInformant to use in this Protocol, if a
-     * NodeInformant is supported. Otherwise, throws
-     * UnsupportedOperationException
-     *
-     * @param nodeInformant
-     */
-    void setNodeInformant(NodeInformant nodeInformant);
-
-    /**
-     * Receives the handshake from the Peer
-     *
-     * @param peer
-     * @throws IOException
-     * @throws HandshakeException
-     */
-    void handshake(Peer peer) throws IOException, HandshakeException;
-
-    /**
-     * Returns <code>true</code> if the handshaking process was completed
-     * successfully, <code>false</code> if either the handshaking process has
-     * not happened or the handshake failed
-     *
-     * @return
-     */
-    boolean isHandshakeSuccessful();
-
-    /**
-     * Negotiates the FlowFileCodec that is to be used for transferring
-     * FlowFiles
-     *
-     * @param peer
-     * @return
-     * @throws IOException
-     * @throws BadRequestException
-     */
-    FlowFileCodec negotiateCodec(Peer peer) throws IOException, ProtocolException;
-
-    /**
-     * Returns the codec that has already been negotiated by this Protocol, if
-     * any.
-     *
-     * @return
-     */
-    FlowFileCodec getPreNegotiatedCodec();
-
-    /**
-     * Reads the Request Type of the next request from the Peer
-     *
-     * @return the RequestType that the peer would like to happen - or null, if
-     * no data available
-     */
-    RequestType getRequestType(Peer peer) throws IOException;
-
-    /**
-     * Sends FlowFiles to the specified peer
-     *
-     * @param peer
-     * @param context
-     * @param session
-     * @param codec
-     *
-     * @return the number of FlowFiles transferred
-     */
-    int transferFlowFiles(Peer peer, ProcessContext context, ProcessSession session, FlowFileCodec codec) throws IOException, ProtocolException;
-
-    /**
-     * Receives FlowFiles from the specified peer
-     *
-     * @param peer
-     * @param context
-     * @param session
-     * @param codec
-     * @throws IOException
-     *
-     * @return the number of FlowFiles received
-     * @throws ProtocolException
-     */
-    int receiveFlowFiles(Peer peer, ProcessContext context, ProcessSession session, FlowFileCodec codec) throws IOException, ProtocolException;
-
-    /**
-     * Returns the number of milliseconds after a request is received for which
-     * the request is still valid. A valid of 0 indicates that the request will
-     * not expire.
-     *
-     * @return
-     */
-    long getRequestExpiration();
-
-    /**
-     * Sends a list of all nodes in the cluster to the specified peer. If not in
-     * a cluster, sends info about itself
-     *
-     * @param peer
-     */
-    void sendPeerList(Peer peer) throws IOException;
-
-    void shutdown(Peer peer);
-
-    boolean isShutdown();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/.gitignore
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/.gitignore b/nar-bundles/framework-bundle/framework/core/.gitignore
deleted file mode 100755
index ea8c4bf..0000000
--- a/nar-bundles/framework-bundle/framework/core/.gitignore
+++ /dev/null
@@ -1 +0,0 @@
-/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/pom.xml b/nar-bundles/framework-bundle/framework/core/pom.xml
deleted file mode 100644
index 47b52ea..0000000
--- a/nar-bundles/framework-bundle/framework/core/pom.xml
+++ /dev/null
@@ -1,121 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-framework-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-    <artifactId>framework-core</artifactId>
-    <packaging>jar</packaging>
-    <name>NiFi Framework Core</name>
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>core-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-expression-language</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-properties</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>site-to-site</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>ch.qos.logback</groupId>
-            <artifactId>logback-classic</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>framework-cluster-protocol</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-logging-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>client-dto</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.quartz-scheduler</groupId>
-            <artifactId>quartz</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.h2database</groupId>
-            <artifactId>h2</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.jasypt</groupId>
-            <artifactId>jasypt</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.bouncycastle</groupId>
-            <artifactId>bcprov-jdk16</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.sun.jersey</groupId>
-            <artifactId>jersey-client</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-security</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-core</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-beans</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-context</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-io</groupId>
-            <artifactId>commons-io</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>data-provenance-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>wali</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-mock</artifactId>
-            <scope>test</scope>
-        </dependency>
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/BulletinsPayload.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/BulletinsPayload.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/BulletinsPayload.java
deleted file mode 100644
index 1249657..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/BulletinsPayload.java
+++ /dev/null
@@ -1,95 +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.cluster;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Set;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.jaxb.BulletinAdapter;
-import org.apache.nifi.reporting.Bulletin;
-
-/**
- * The payload of the bulletins.
- *
- * @author unattributed
- */
-@XmlRootElement
-public class BulletinsPayload {
-
-    private static final JAXBContext JAXB_CONTEXT;
-
-    static {
-        try {
-            JAXB_CONTEXT = JAXBContext.newInstance(BulletinsPayload.class);
-        } catch (JAXBException e) {
-            throw new RuntimeException("Unable to create JAXBContext.");
-        }
-    }
-
-    private Set<Bulletin> bulletins;
-
-    @XmlJavaTypeAdapter(BulletinAdapter.class)
-    public Set<Bulletin> getBulletins() {
-        return bulletins;
-    }
-
-    public void setBulletins(final Set<Bulletin> bulletins) {
-        this.bulletins = bulletins;
-    }
-
-    public byte[] marshal() throws ProtocolException {
-        final ByteArrayOutputStream payloadBytes = new ByteArrayOutputStream();
-        marshal(this, payloadBytes);
-        return payloadBytes.toByteArray();
-    }
-
-    public static void marshal(final BulletinsPayload payload, final OutputStream os) throws ProtocolException {
-        try {
-            final Marshaller marshaller = JAXB_CONTEXT.createMarshaller();
-            marshaller.marshal(payload, os);
-        } catch (final JAXBException je) {
-            throw new ProtocolException(je);
-        }
-    }
-
-    public static BulletinsPayload unmarshal(final InputStream is) throws ProtocolException {
-        try {
-            final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
-            return (BulletinsPayload) unmarshaller.unmarshal(is);
-        } catch (final JAXBException je) {
-            throw new ProtocolException(je);
-        }
-    }
-
-    public static BulletinsPayload unmarshal(final byte[] bytes) throws ProtocolException {
-        try {
-            final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
-            return (BulletinsPayload) unmarshaller.unmarshal(new ByteArrayInputStream(bytes));
-        } catch (final JAXBException je) {
-            throw new ProtocolException(je);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/ConnectionException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/ConnectionException.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/ConnectionException.java
deleted file mode 100644
index 986e904..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/ConnectionException.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.cluster;
-
-/**
- * Represents the exceptional case when connection to the cluster fails.
- *
- * @author unattributed
- */
-public class ConnectionException extends RuntimeException {
-
-    private static final long serialVersionUID = -1378294897231234028L;
-
-    public ConnectionException() {
-    }
-
-    public ConnectionException(String msg) {
-        super(msg);
-    }
-
-    public ConnectionException(Throwable cause) {
-        super(cause);
-    }
-
-    public ConnectionException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/DisconnectionException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/DisconnectionException.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/DisconnectionException.java
deleted file mode 100644
index 55707f3..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/DisconnectionException.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.cluster;
-
-/**
- * Represents the exceptional case when disconnection from the cluster fails.
- *
- * @author unattributed
- */
-public class DisconnectionException extends RuntimeException {
-
-    private static final long serialVersionUID = 6648876367997026125L;
-
-    public DisconnectionException() {
-    }
-
-    public DisconnectionException(String msg) {
-        super(msg);
-    }
-
-    public DisconnectionException(Throwable cause) {
-        super(cause);
-    }
-
-    public DisconnectionException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/HeartbeatPayload.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/HeartbeatPayload.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/HeartbeatPayload.java
deleted file mode 100644
index 093b238..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/HeartbeatPayload.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.cluster;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.List;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.controller.Counter;
-import org.apache.nifi.controller.status.ProcessGroupStatus;
-import org.apache.nifi.diagnostics.SystemDiagnostics;
-import org.apache.nifi.jaxb.CounterAdapter;
-
-/**
- * The payload of the heartbeat. The payload contains status to inform the
- * cluster manager the current workload of this node.
- *
- * @author unattributed
- */
-@XmlRootElement
-public class HeartbeatPayload {
-
-    private static final JAXBContext JAXB_CONTEXT;
-
-    static {
-        try {
-            JAXB_CONTEXT = JAXBContext.newInstance(HeartbeatPayload.class);
-        } catch (JAXBException e) {
-            throw new RuntimeException("Unable to create JAXBContext.");
-        }
-    }
-
-    private List<Counter> counters;
-    private ProcessGroupStatus processGroupStatus;
-    private int activeThreadCount;
-    private long totalFlowFileCount;
-    private long totalFlowFileBytes;
-    private SystemDiagnostics systemDiagnostics;
-    private Integer siteToSitePort;
-    private boolean siteToSiteSecure;
-    private long systemStartTime;
-
-    @XmlJavaTypeAdapter(CounterAdapter.class)
-    public List<Counter> getCounters() {
-        return counters;
-    }
-
-    public void setCounters(final List<Counter> counters) {
-        this.counters = counters;
-    }
-
-    public int getActiveThreadCount() {
-        return activeThreadCount;
-    }
-
-    public void setActiveThreadCount(final int activeThreadCount) {
-        this.activeThreadCount = activeThreadCount;
-    }
-
-    public long getTotalFlowFileCount() {
-        return totalFlowFileCount;
-    }
-
-    public void setTotalFlowFileCount(final long totalFlowFileCount) {
-        this.totalFlowFileCount = totalFlowFileCount;
-    }
-
-    public long getTotalFlowFileBytes() {
-        return totalFlowFileBytes;
-    }
-
-    public void setTotalFlowFileBytes(final long totalFlowFileBytes) {
-        this.totalFlowFileBytes = totalFlowFileBytes;
-    }
-
-    public ProcessGroupStatus getProcessGroupStatus() {
-        return processGroupStatus;
-    }
-
-    public void setProcessGroupStatus(final ProcessGroupStatus processGroupStatus) {
-        this.processGroupStatus = processGroupStatus;
-    }
-
-    public SystemDiagnostics getSystemDiagnostics() {
-        return systemDiagnostics;
-    }
-
-    public void setSystemDiagnostics(final SystemDiagnostics systemDiagnostics) {
-        this.systemDiagnostics = systemDiagnostics;
-    }
-
-    public boolean isSiteToSiteSecure() {
-        return siteToSiteSecure;
-    }
-
-    public void setSiteToSiteSecure(final boolean secure) {
-        this.siteToSiteSecure = secure;
-    }
-
-    public Integer getSiteToSitePort() {
-        return siteToSitePort;
-    }
-
-    public void setSiteToSitePort(final Integer port) {
-        this.siteToSitePort = port;
-    }
-
-    public long getSystemStartTime() {
-        return systemStartTime;
-    }
-
-    public void setSystemStartTime(final long systemStartTime) {
-        this.systemStartTime = systemStartTime;
-    }
-
-    public byte[] marshal() throws ProtocolException {
-        final ByteArrayOutputStream payloadBytes = new ByteArrayOutputStream();
-        marshal(this, payloadBytes);
-        return payloadBytes.toByteArray();
-    }
-
-    public static void marshal(final HeartbeatPayload payload, final OutputStream os) throws ProtocolException {
-        try {
-            final Marshaller marshaller = JAXB_CONTEXT.createMarshaller();
-            marshaller.marshal(payload, os);
-        } catch (final JAXBException je) {
-            throw new ProtocolException(je);
-        }
-    }
-
-    public static HeartbeatPayload unmarshal(final InputStream is) throws ProtocolException {
-        try {
-            final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
-            return (HeartbeatPayload) unmarshaller.unmarshal(is);
-        } catch (final JAXBException je) {
-            throw new ProtocolException(je);
-        }
-    }
-
-    public static HeartbeatPayload unmarshal(final byte[] bytes) throws ProtocolException {
-        try {
-            final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
-            return (HeartbeatPayload) unmarshaller.unmarshal(new ByteArrayInputStream(bytes));
-        } catch (final JAXBException je) {
-            throw new ProtocolException(je);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/connectable/LocalPort.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/connectable/LocalPort.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/connectable/LocalPort.java
deleted file mode 100644
index f0739c2..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/connectable/LocalPort.java
+++ /dev/null
@@ -1,172 +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.connectable;
-
-import org.apache.nifi.connectable.ConnectableType;
-import org.apache.nifi.connectable.Connection;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.controller.AbstractPort;
-import org.apache.nifi.controller.ProcessScheduler;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.groups.ProcessGroup;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-
-/**
- * Provides a mechanism by which <code>FlowFile</code>s can be transferred into
- * and out of a <code>ProcessGroup</code> to and/or from another
- * <code>ProcessGroup</code> within the same instance of NiFi.
- */
-public class LocalPort extends AbstractPort {
-
-    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
-    private final Lock readLock = rwLock.readLock();
-    private final Lock writeLock = rwLock.writeLock();
-
-    public LocalPort(final String id, final String name, final ProcessGroup processGroup, final ConnectableType type, final ProcessScheduler scheduler) {
-        super(id, name, processGroup, type, scheduler);
-    }
-
-    @Override
-    public boolean isValid() {
-        return !getConnections(Relationship.ANONYMOUS).isEmpty();
-    }
-
-    @Override
-    public Collection<ValidationResult> getValidationErrors() {
-        final Collection<ValidationResult> validationErrors = new ArrayList<>();
-        if (!isValid()) {
-            final ValidationResult error = new ValidationResult.Builder()
-                    .explanation(String.format("Output connection for port '%s' is not defined.", getName()))
-                    .subject(String.format("Port '%s'", getName()))
-                    .valid(false)
-                    .build();
-            validationErrors.add(error);
-        }
-        return validationErrors;
-    }
-
-    @Override
-    public void onTrigger(final ProcessContext context, final ProcessSession session) {
-        readLock.lock();
-        try {
-            final List<FlowFile> flowFiles = session.get(10);
-            if (flowFiles.isEmpty()) {
-                context.yield();
-            } else {
-                session.transfer(flowFiles, Relationship.ANONYMOUS);
-            }
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public void updateConnection(final Connection connection) throws IllegalStateException {
-        writeLock.lock();
-        try {
-            super.updateConnection(connection);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public void addConnection(final Connection connection) throws IllegalArgumentException {
-        writeLock.lock();
-        try {
-            super.addConnection(connection);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public void removeConnection(final Connection connection) throws IllegalArgumentException, IllegalStateException {
-        writeLock.lock();
-        try {
-            super.removeConnection(connection);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public Set<Connection> getConnections() {
-        readLock.lock();
-        try {
-            return super.getConnections();
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public Set<Connection> getConnections(Relationship relationship) {
-        readLock.lock();
-        try {
-            return super.getConnections(relationship);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public List<Connection> getIncomingConnections() {
-        readLock.lock();
-        try {
-            return super.getIncomingConnections();
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public boolean hasIncomingConnection() {
-        readLock.lock();
-        try {
-            return super.hasIncomingConnection();
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public boolean isTriggerWhenEmpty() {
-        return false;
-    }
-
-    @Override
-    public SchedulingStrategy getSchedulingStrategy() {
-        return SchedulingStrategy.TIMER_DRIVEN;
-    }
-
-    @Override
-    public boolean isSideEffectFree() {
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/connectable/StandardConnection.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/connectable/StandardConnection.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/connectable/StandardConnection.java
deleted file mode 100644
index 1d723b5..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/connectable/StandardConnection.java
+++ /dev/null
@@ -1,336 +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.connectable;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.nifi.controller.FlowFileQueue;
-import org.apache.nifi.controller.ProcessScheduler;
-import org.apache.nifi.controller.StandardFlowFileQueue;
-import org.apache.nifi.controller.repository.FlowFileRecord;
-import org.apache.nifi.groups.ProcessGroup;
-import org.apache.nifi.processor.FlowFileFilter;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.util.NiFiProperties;
-
-import org.apache.commons.lang3.builder.EqualsBuilder;
-import org.apache.commons.lang3.builder.HashCodeBuilder;
-
-/**
- * Models a connection between connectable components. A connection may contain
- * one or more relationships that map the source component to the destination
- * component.
- */
-public final class StandardConnection implements Connection {
-
-    private final String id;
-    private final AtomicReference<ProcessGroup> processGroup;
-    private final AtomicReference<String> name;
-    private final AtomicReference<List<Position>> bendPoints;
-    private final Connectable source;
-    private final AtomicReference<Connectable> destination;
-    private final AtomicReference<Collection<Relationship>> relationships;
-    private final StandardFlowFileQueue flowFileQueue;
-    private final AtomicInteger labelIndex = new AtomicInteger(1);
-    private final AtomicLong zIndex = new AtomicLong(0L);
-    private final ProcessScheduler scheduler;
-    private final int hashCode;
-
-    private StandardConnection(final Builder builder) {
-        id = builder.id;
-        name = new AtomicReference<>(builder.name);
-        bendPoints = new AtomicReference<>(Collections.unmodifiableList(new ArrayList<>(builder.bendPoints)));
-        processGroup = new AtomicReference<>(builder.processGroup);
-        source = builder.source;
-        destination = new AtomicReference<>(builder.destination);
-        relationships = new AtomicReference<>(Collections.unmodifiableCollection(builder.relationships));
-        scheduler = builder.scheduler;
-        flowFileQueue = new StandardFlowFileQueue(id, this, scheduler, NiFiProperties.getInstance().getQueueSwapThreshold());
-        hashCode = new HashCodeBuilder(7, 67).append(id).toHashCode();
-    }
-
-    public ProcessGroup getProcessGroup() {
-        return processGroup.get();
-    }
-
-    public String getIdentifier() {
-        return id;
-    }
-
-    public String getName() {
-        return name.get();
-    }
-
-    public void setName(final String name) {
-        this.name.set(name);
-    }
-
-    @Override
-    public List<Position> getBendPoints() {
-        return bendPoints.get();
-    }
-
-    @Override
-    public void setBendPoints(final List<Position> position) {
-        this.bendPoints.set(Collections.unmodifiableList(new ArrayList<>(position)));
-    }
-
-    public int getLabelIndex() {
-        return labelIndex.get();
-    }
-
-    public void setLabelIndex(final int labelIndex) {
-        this.labelIndex.set(labelIndex);
-    }
-
-    @Override
-    public long getZIndex() {
-        return zIndex.get();
-    }
-
-    @Override
-    public void setZIndex(final long zIndex) {
-        this.zIndex.set(zIndex);
-    }
-
-    public Connectable getSource() {
-        return source;
-    }
-
-    public Connectable getDestination() {
-        return destination.get();
-    }
-
-    public Collection<Relationship> getRelationships() {
-        return relationships.get();
-    }
-
-    public FlowFileQueue getFlowFileQueue() {
-        return flowFileQueue;
-    }
-
-    public void setProcessGroup(final ProcessGroup newGroup) {
-        final ProcessGroup currentGroup = this.processGroup.get();
-        try {
-            this.processGroup.set(newGroup);
-        } catch (final RuntimeException e) {
-            this.processGroup.set(currentGroup);
-            throw e;
-        }
-    }
-
-    public void setRelationships(final Collection<Relationship> newRelationships) {
-        final Collection<Relationship> currentRelationships = relationships.get();
-        if (currentRelationships.equals(newRelationships)) {
-            return;
-        }
-
-        if (getSource().isRunning()) {
-            throw new IllegalStateException("Cannot update the relationships for Connection because the source of the Connection is running");
-        }
-
-        try {
-            this.relationships.set(new ArrayList<>(newRelationships));
-            getSource().updateConnection(this);
-        } catch (final RuntimeException e) {
-            this.relationships.set(currentRelationships);
-            throw e;
-        }
-    }
-
-    public void setDestination(final Connectable newDestination) {
-        final Connectable previousDestination = destination.get();
-        if (previousDestination.equals(newDestination)) {
-            return;
-        }
-
-        if (previousDestination.isRunning() && !(previousDestination instanceof Funnel || previousDestination instanceof LocalPort)) {
-            throw new IllegalStateException("Cannot change destination of Connection because the current destination is running");
-        }
-
-        try {
-            previousDestination.removeConnection(this);
-            this.destination.set(newDestination);
-            getSource().updateConnection(this);
-
-            newDestination.addConnection(this);
-            scheduler.registerEvent(newDestination);
-        } catch (final RuntimeException e) {
-            this.destination.set(previousDestination);
-            throw e;
-        }
-    }
-
-    @Override
-    public void lock() {
-        flowFileQueue.lock();
-    }
-
-    @Override
-    public void unlock() {
-        flowFileQueue.unlock();
-    }
-
-    @Override
-    public List<FlowFileRecord> poll(final FlowFileFilter filter, final Set<FlowFileRecord> expiredRecords) {
-        return flowFileQueue.poll(filter, expiredRecords);
-    }
-
-    @Override
-    public boolean equals(final Object other) {
-        if (!(other instanceof Connection)) {
-            return false;
-        }
-        final Connection con = (Connection) other;
-        return new EqualsBuilder().append(id, con.getIdentifier()).isEquals();
-    }
-
-    @Override
-    public int hashCode() {
-        return hashCode;
-    }
-
-    @Override
-    public String toString() {
-        return "Connection[ID=" + id + ",Name=" + name.get() + ",Source=" + getSource() + ",Destination=" + getDestination() + ",Relationships=" + getRelationships();
-    }
-
-    /**
-     * Gives this Connection ownership of the given FlowFile and allows the
-     * Connection to hold on to the FlowFile but NOT provide the FlowFile to
-     * consumers. This allows us to ensure that the Connection is not deleted
-     * during the middle of a Session commit.
-     *
-     * @param flowFile
-     */
-    @Override
-    public void enqueue(final FlowFileRecord flowFile) {
-        flowFileQueue.put(flowFile);
-    }
-
-    @Override
-    public void enqueue(final Collection<FlowFileRecord> flowFiles) {
-        flowFileQueue.putAll(flowFiles);
-    }
-
-    public static class Builder {
-
-        private final ProcessScheduler scheduler;
-
-        private String id = UUID.randomUUID().toString();
-        private String name;
-        private List<Position> bendPoints = new ArrayList<>();
-        private ProcessGroup processGroup;
-        private Connectable source;
-        private Connectable destination;
-        private Collection<Relationship> relationships;
-
-        public Builder(final ProcessScheduler scheduler) {
-            this.scheduler = scheduler;
-        }
-
-        public Builder id(final String id) {
-            this.id = id;
-            return this;
-        }
-
-        public Builder source(final Connectable source) {
-            this.source = source;
-            return this;
-        }
-
-        public Builder processGroup(final ProcessGroup group) {
-            this.processGroup = group;
-            return this;
-        }
-
-        public Builder destination(final Connectable destination) {
-            this.destination = destination;
-            return this;
-        }
-
-        public Builder relationships(final Collection<Relationship> relationships) {
-            this.relationships = new ArrayList<>(relationships);
-            return this;
-        }
-
-        public Builder name(final String name) {
-            this.name = name;
-            return this;
-        }
-
-        public Builder bendPoints(final List<Position> bendPoints) {
-            this.bendPoints.clear();
-            this.bendPoints.addAll(bendPoints);
-            return this;
-        }
-
-        public Builder addBendPoint(final Position bendPoint) {
-            bendPoints.add(bendPoint);
-            return this;
-        }
-
-        public StandardConnection build() {
-            if (source == null) {
-                throw new IllegalStateException("Cannot build a Connection without a Source");
-            }
-            if (destination == null) {
-                throw new IllegalStateException("Cannot build a Connection without a Destination");
-            }
-
-            if (relationships == null) {
-                relationships = new ArrayList<>();
-            }
-
-            if (relationships.isEmpty()) {
-                // ensure relationships have been specified for processors, otherwise the anonymous relationship is used
-                if (source.getConnectableType() == ConnectableType.PROCESSOR) {
-                    throw new IllegalStateException("Cannot build a Connection without any relationships");
-                }
-                relationships.add(Relationship.ANONYMOUS);
-            }
-
-            return new StandardConnection(this);
-        }
-    }
-
-    @Override
-    public void verifyCanUpdate() {
-        // StandardConnection can always be updated
-    }
-
-    @Override
-    public void verifyCanDelete() {
-        if (!flowFileQueue.isEmpty()) {
-            throw new IllegalStateException("Queue not empty for " + this);
-        }
-
-        if (source.isRunning()) {
-            if (!ConnectableType.FUNNEL.equals(source.getConnectableType())) {
-                throw new IllegalStateException("Source of Connection (" + source + ") is running");
-            }
-        }
-    }
-}


[23/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceOptionsDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceOptionsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceOptionsDTO.java
deleted file mode 100644
index 7f939b9..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceOptionsDTO.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.web.api.dto.provenance;
-
-import java.util.List;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * A provenance search options.
- */
-@XmlType(name = "provenanceOptions")
-public class ProvenanceOptionsDTO {
-
-    private List<ProvenanceSearchableFieldDTO> searchableFields;
-
-    /**
-     * The available searchable fields for this NiFi instance.
-     *
-     * @return
-     */
-    public List<ProvenanceSearchableFieldDTO> getSearchableFields() {
-        return searchableFields;
-    }
-
-    public void setSearchableFields(List<ProvenanceSearchableFieldDTO> searchableFields) {
-        this.searchableFields = searchableFields;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceRequestDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceRequestDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceRequestDTO.java
deleted file mode 100644
index e9e5453..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceRequestDTO.java
+++ /dev/null
@@ -1,118 +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.api.dto.provenance;
-
-import java.util.Date;
-import java.util.Map;
-
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.DateTimeAdapter;
-
-/**
- * A request for provenance.
- */
-@XmlType(name = "provenanceRequest")
-public class ProvenanceRequestDTO {
-
-    private Map<String, String> searchTerms;
-    private Date startDate;
-    private Date endDate;
-    private String minimumFileSize;
-    private String maximumFileSize;
-    private Integer maxResults;
-
-    /**
-     * Returns the search terms to use for this search
-     *
-     * @return
-     */
-    public Map<String, String> getSearchTerms() {
-        return searchTerms;
-    }
-
-    public void setSearchTerms(final Map<String, String> searchTerms) {
-        this.searchTerms = searchTerms;
-    }
-
-    /**
-     * The earliest event time to include in the query
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(DateTimeAdapter.class)
-    public Date getStartDate() {
-        return startDate;
-    }
-
-    public void setStartDate(Date startDate) {
-        this.startDate = startDate;
-    }
-
-    /**
-     * The latest event time to include in the query
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(DateTimeAdapter.class)
-    public Date getEndDate() {
-        return endDate;
-    }
-
-    public void setEndDate(Date endDate) {
-        this.endDate = endDate;
-    }
-
-    /**
-     * The minimum file size to include in the query.
-     *
-     * @return
-     */
-    public String getMinimumFileSize() {
-        return minimumFileSize;
-    }
-
-    public void setMinimumFileSize(String minimumFileSize) {
-        this.minimumFileSize = minimumFileSize;
-    }
-
-    /**
-     * The maximum file size to include in the query.
-     *
-     * @return
-     */
-    public String getMaximumFileSize() {
-        return maximumFileSize;
-    }
-
-    public void setMaximumFileSize(String maximumFileSize) {
-        this.maximumFileSize = maximumFileSize;
-    }
-
-    /**
-     * The number of max results.
-     *
-     * @return
-     */
-    public Integer getMaxResults() {
-        return maxResults;
-    }
-
-    public void setMaxResults(Integer maxResults) {
-        this.maxResults = maxResults;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceResultsDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceResultsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceResultsDTO.java
deleted file mode 100644
index d6ea8a0..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceResultsDTO.java
+++ /dev/null
@@ -1,136 +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.api.dto.provenance;
-
-import org.apache.nifi.web.api.dto.util.DateTimeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import java.util.Date;
-import java.util.List;
-import java.util.Set;
-
-/**
- * Results of a provenance request.
- */
-@XmlType(name = "provenanceResults")
-public class ProvenanceResultsDTO {
-
-    private List<ProvenanceEventDTO> provenanceEvents;
-    private String total;
-    private Long totalCount;
-    private Date generated;
-    private Date oldestEvent;
-    private Integer timeOffset;
-
-    private Set<String> errors;
-
-    /**
-     * Any error messages.
-     *
-     * @return
-     */
-    public Set<String> getErrors() {
-        return errors;
-    }
-
-    public void setErrors(Set<String> errors) {
-        this.errors = errors;
-    }
-
-    /**
-     * The provenance events that matched the search criteria.
-     *
-     * @return
-     */
-    public List<ProvenanceEventDTO> getProvenanceEvents() {
-        return provenanceEvents;
-    }
-
-    public void setProvenanceEvents(List<ProvenanceEventDTO> provenanceEvents) {
-        this.provenanceEvents = provenanceEvents;
-    }
-
-    /**
-     * The total number of results formatted.
-     *
-     * @return
-     */
-    public String getTotal() {
-        return total;
-    }
-
-    public void setTotal(String total) {
-        this.total = total;
-    }
-
-    /**
-     * The total number of results.
-     *
-     * @return
-     */
-    public Long getTotalCount() {
-        return totalCount;
-    }
-
-    public void setTotalCount(Long totalCount) {
-        this.totalCount = totalCount;
-    }
-
-    /**
-     * When the search was performed.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    public Date getGenerated() {
-        return generated;
-    }
-
-    public void setGenerated(Date generated) {
-        this.generated = generated;
-    }
-
-    /**
-     * The oldest event available in the provenance repository.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(DateTimeAdapter.class)
-    public Date getOldestEvent() {
-        return oldestEvent;
-    }
-
-    public void setOldestEvent(Date oldestEvent) {
-        this.oldestEvent = oldestEvent;
-    }
-
-    /**
-     * The time offset on the server thats used for event time.
-     *
-     * @return
-     */
-    public Integer getTimeOffset() {
-        return timeOffset;
-    }
-
-    public void setTimeOffset(Integer timeOffset) {
-        this.timeOffset = timeOffset;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceSearchableFieldDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceSearchableFieldDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceSearchableFieldDTO.java
deleted file mode 100644
index af89f97..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceSearchableFieldDTO.java
+++ /dev/null
@@ -1,84 +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.api.dto.provenance;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * A searchable field for provenance queries.
- */
-@XmlType(name = "provenanceSearchableField")
-public class ProvenanceSearchableFieldDTO {
-
-    private String id;
-    private String field;
-    private String label;
-    private String type;
-
-    /**
-     * The id of this searchable field.
-     *
-     * @return
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The field.
-     *
-     * @return
-     */
-    public String getField() {
-        return field;
-    }
-
-    public void setField(String field) {
-        this.field = field;
-    }
-
-    /**
-     * The label for this field.
-     *
-     * @return
-     */
-    public String getLabel() {
-        return label;
-    }
-
-    public void setLabel(String label) {
-        this.label = label;
-    }
-
-    /**
-     * The type for this field.
-     *
-     * @return
-     */
-    public String getType() {
-        return type;
-    }
-
-    public void setType(String type) {
-        this.type = type;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageDTO.java
deleted file mode 100644
index 745105e..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageDTO.java
+++ /dev/null
@@ -1,161 +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.api.dto.provenance.lineage;
-
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimestampAdapter;
-
-/**
- * Represents the lineage for a flowfile.
- */
-@XmlType(name = "lineage")
-public class LineageDTO {
-
-    private String id;
-    private String uri;
-    private String clusterNodeId;
-
-    private Date submissionTime;
-    private Date expiration;
-    private Integer percentCompleted;
-    private Boolean finished;
-
-    private LineageRequestDTO request;
-    private LineageResultsDTO results;
-
-    /**
-     * The id of this lineage.
-     *
-     * @return
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The uri for this lineage.
-     *
-     * @return
-     */
-    public String getUri() {
-        return uri;
-    }
-
-    public void setUri(String uri) {
-        this.uri = uri;
-    }
-
-    /**
-     * The id of the node in the cluster where this lineage originated.
-     *
-     * @return
-     */
-    public String getClusterNodeId() {
-        return clusterNodeId;
-    }
-
-    public void setClusterNodeId(String clusterNodeId) {
-        this.clusterNodeId = clusterNodeId;
-    }
-
-    /**
-     * The submission time for this lineage.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimestampAdapter.class)
-    public Date getSubmissionTime() {
-        return submissionTime;
-    }
-
-    public void setSubmissionTime(Date submissionTime) {
-        this.submissionTime = submissionTime;
-    }
-
-    /**
-     * The expiration of this lineage.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimestampAdapter.class)
-    public Date getExpiration() {
-        return expiration;
-    }
-
-    public void setExpiration(Date expiration) {
-        this.expiration = expiration;
-    }
-
-    /**
-     * Percent completed for this result.
-     *
-     * @return
-     */
-    public Integer getPercentCompleted() {
-        return percentCompleted;
-    }
-
-    public void setPercentCompleted(Integer percentCompleted) {
-        this.percentCompleted = percentCompleted;
-    }
-
-    /**
-     * Whether or not the request is finished running.
-     *
-     * @return
-     */
-    public Boolean getFinished() {
-        return finished;
-    }
-
-    public void setFinished(Boolean finished) {
-        this.finished = finished;
-    }
-
-    /**
-     * The lineage request.
-     *
-     * @return
-     */
-    public LineageRequestDTO getRequest() {
-        return request;
-    }
-
-    public void setRequest(LineageRequestDTO request) {
-        this.request = request;
-    }
-
-    /**
-     * The results of this lineage.
-     *
-     * @return
-     */
-    public LineageResultsDTO getResults() {
-        return results;
-    }
-
-    public void setResults(LineageResultsDTO results) {
-        this.results = results;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageRequestDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageRequestDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageRequestDTO.java
deleted file mode 100644
index 23cf892..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageRequestDTO.java
+++ /dev/null
@@ -1,88 +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.api.dto.provenance.lineage;
-
-import javax.xml.bind.annotation.XmlEnum;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Represents the request for lineage for a flowfile.
- */
-@XmlType(name = "lineageRequest")
-public class LineageRequestDTO {
-
-    /**
-     * The type of this lineage request.
-     */
-    @XmlType(name = "lineageRequestType")
-    @XmlEnum
-    public enum LineageRequestType {
-
-        PARENTS,
-        CHILDREN,
-        FLOWFILE;
-    };
-
-    private Long eventId;
-    private LineageRequestType lineageRequestType;
-
-    private String uuid;
-
-    /**
-     * The event id that was used to generate this lineage.
-     *
-     * @return
-     */
-    public Long getEventId() {
-        return eventId;
-    }
-
-    public void setEventId(Long eventId) {
-        this.eventId = eventId;
-    }
-
-    /**
-     * The type of lineage request. Either 'PARENTS', 'CHILDREN', or 'FLOWFILE'.
-     * PARENTS will return the lineage for the flowfiles that are parents of the
-     * specified event. CHILDREN will return the lineage of for the flowfiles
-     * that are children of the specified event. FLOWFILE will return the
-     * lineage for the specified flowfile.
-     *
-     * @return
-     */
-    public LineageRequestType getLineageRequestType() {
-        return lineageRequestType;
-    }
-
-    public void setLineageRequestType(LineageRequestType lineageRequestType) {
-        this.lineageRequestType = lineageRequestType;
-    }
-
-    /**
-     * The uuid that was used to generate this lineage.
-     *
-     * @return
-     */
-    public String getUuid() {
-        return uuid;
-    }
-
-    public void setUuid(String uuid) {
-        this.uuid = uuid;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageResultsDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageResultsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageResultsDTO.java
deleted file mode 100644
index 77b6e7a..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageResultsDTO.java
+++ /dev/null
@@ -1,73 +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.api.dto.provenance.lineage;
-
-import java.util.List;
-import java.util.Set;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Represents the lineage results.
- */
-@XmlType(name = "lineage")
-public class LineageResultsDTO {
-
-    private Set<String> errors;
-
-    private List<ProvenanceNodeDTO> nodes;
-    private List<ProvenanceLinkDTO> links;
-
-    /**
-     * Any error messages.
-     *
-     * @return
-     */
-    public Set<String> getErrors() {
-        return errors;
-    }
-
-    public void setErrors(Set<String> errors) {
-        this.errors = errors;
-    }
-
-    /**
-     * The nodes.
-     *
-     * @return
-     */
-    public List<ProvenanceNodeDTO> getNodes() {
-        return nodes;
-    }
-
-    public void setNodes(List<ProvenanceNodeDTO> nodes) {
-        this.nodes = nodes;
-    }
-
-    /**
-     * The links.
-     *
-     * @return
-     */
-    public List<ProvenanceLinkDTO> getLinks() {
-        return links;
-    }
-
-    public void setLinks(List<ProvenanceLinkDTO> links) {
-        this.links = links;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceLinkDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceLinkDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceLinkDTO.java
deleted file mode 100644
index f3bbcef..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceLinkDTO.java
+++ /dev/null
@@ -1,101 +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.api.dto.provenance.lineage;
-
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimestampAdapter;
-
-/**
- * A link between an event or flowfile within a provenance lineage.
- */
-@XmlType(name = "provenanceLink")
-public class ProvenanceLinkDTO {
-
-    private String sourceId;
-    private String targetId;
-    private String flowFileUuid;
-    private Date timestamp;
-    private Long millis;
-
-    /**
-     * The source node id.
-     *
-     * @return
-     */
-    public String getSourceId() {
-        return sourceId;
-    }
-
-    public void setSourceId(String sourceId) {
-        this.sourceId = sourceId;
-    }
-
-    /**
-     * The target node id.
-     *
-     * @return
-     */
-    public String getTargetId() {
-        return targetId;
-    }
-
-    public void setTargetId(String targetId) {
-        this.targetId = targetId;
-    }
-
-    /**
-     * The flowfile uuid that traversed this link.
-     *
-     * @return
-     */
-    public String getFlowFileUuid() {
-        return flowFileUuid;
-    }
-
-    public void setFlowFileUuid(String flowFileUuid) {
-        this.flowFileUuid = flowFileUuid;
-    }
-
-    /**
-     * The timestamp of this link (based on the destination).
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimestampAdapter.class)
-    public Date getTimestamp() {
-        return timestamp;
-    }
-
-    public void setTimestamp(Date timestamp) {
-        this.timestamp = timestamp;
-    }
-
-    /**
-     * The number of millis since epoch.
-     *
-     * @return
-     */
-    public Long getMillis() {
-        return millis;
-    }
-
-    public void setMillis(Long millis) {
-        this.millis = millis;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceNodeDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceNodeDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceNodeDTO.java
deleted file mode 100644
index c15c598..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceNodeDTO.java
+++ /dev/null
@@ -1,162 +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.api.dto.provenance.lineage;
-
-import java.util.Date;
-import java.util.List;
-
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-
-import org.apache.nifi.web.api.dto.util.TimestampAdapter;
-
-/**
- * A node within a provenance lineage. May represent either an event or a
- * flowfile.
- */
-@XmlType(name = "provenanceNode")
-public class ProvenanceNodeDTO {
-
-    private String id;
-    private String flowFileUuid;
-    private List<String> parentUuids;
-    private List<String> childUuids;
-    private String clusterNodeIdentifier;
-    private String type;
-    private String eventType;
-    private Long millis;
-    private Date timestamp;
-
-    /**
-     * The id of the node.
-     *
-     * @return
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The flowfile uuid for this provenance event.
-     *
-     * @return
-     */
-    public String getFlowFileUuid() {
-        return flowFileUuid;
-    }
-
-    public void setFlowFileUuid(String flowFileUuid) {
-        this.flowFileUuid = flowFileUuid;
-    }
-
-    /**
-     * The parent flowfile uuids for this provenance event.
-     *
-     * @return
-     */
-    public List<String> getParentUuids() {
-        return parentUuids;
-    }
-
-    public void setParentUuids(List<String> parentUuids) {
-        this.parentUuids = parentUuids;
-    }
-
-    /**
-     * The child flowfile uuids for this provenance event.
-     *
-     * @return
-     */
-    public List<String> getChildUuids() {
-        return childUuids;
-    }
-
-    public void setChildUuids(List<String> childUuids) {
-        this.childUuids = childUuids;
-    }
-
-    /**
-     * The node identifier that this event/flowfile originated from.
-     *
-     * @return
-     */
-    public String getClusterNodeIdentifier() {
-        return clusterNodeIdentifier;
-    }
-
-    public void setClusterNodeIdentifier(String clusterNodeIdentifier) {
-        this.clusterNodeIdentifier = clusterNodeIdentifier;
-    }
-
-    /**
-     * The type of node.
-     *
-     * @return
-     */
-    public String getType() {
-        return type;
-    }
-
-    public void setType(String type) {
-        this.type = type;
-    }
-
-    /**
-     * If this is an event node, this is the type of event.
-     *
-     * @return
-     */
-    public String getEventType() {
-        return eventType;
-    }
-
-    public void setEventType(String eventType) {
-        this.eventType = eventType;
-    }
-
-    /**
-     * The timestamp of this node.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimestampAdapter.class)
-    public Date getTimestamp() {
-        return timestamp;
-    }
-
-    public void setTimestamp(Date timestamp) {
-        this.timestamp = timestamp;
-    }
-
-    /**
-     * The number of millis since epoch.
-     *
-     * @return
-     */
-    public Long getMillis() {
-        return millis;
-    }
-
-    public void setMillis(Long millis) {
-        this.millis = millis;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/ComponentSearchResultDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/ComponentSearchResultDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/ComponentSearchResultDTO.java
deleted file mode 100644
index 83d7a91..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/ComponentSearchResultDTO.java
+++ /dev/null
@@ -1,85 +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.api.dto.search;
-
-import java.util.List;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * The components that match a search performed on this NiFi.
- */
-@XmlType(name = "componentSearchResult")
-public class ComponentSearchResultDTO {
-
-    private String id;
-    private String groupId;
-    private String name;
-    private List<String> matches;
-
-    /**
-     * The id of the component that matched.
-     *
-     * @return
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The group id of the component that matched.
-     *
-     * @return
-     */
-    public String getGroupId() {
-        return groupId;
-    }
-
-    public void setGroupId(String groupId) {
-        this.groupId = groupId;
-    }
-
-    /**
-     * The name of the component that matched.
-     *
-     * @return
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * What matched the search string for this component.
-     *
-     * @return
-     */
-    public List<String> getMatches() {
-        return matches;
-    }
-
-    public void setMatches(List<String> matches) {
-        this.matches = matches;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/NodeSearchResultDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/NodeSearchResultDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/NodeSearchResultDTO.java
deleted file mode 100644
index 7c250ee..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/NodeSearchResultDTO.java
+++ /dev/null
@@ -1,56 +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.api.dto.search;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * A search result for a matching node.
- */
-@XmlType(name = "nodeSearchResult")
-public class NodeSearchResultDTO {
-
-    private String id;
-    private String address;
-
-    /**
-     * The id of the node that was matched.
-     *
-     * @return
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The address of the node that was matched.
-     *
-     * @return
-     */
-    public String getAddress() {
-        return address;
-    }
-
-    public void setAddress(String address) {
-        this.address = address;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/SearchResultsDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/SearchResultsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/SearchResultsDTO.java
deleted file mode 100644
index 212d4f2..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/SearchResultsDTO.java
+++ /dev/null
@@ -1,128 +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.api.dto.search;
-
-import java.util.ArrayList;
-import java.util.List;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * The results of a search on this NiFi.
- */
-@XmlType(name = "searchResults")
-public class SearchResultsDTO {
-
-    private List<ComponentSearchResultDTO> processorResults = new ArrayList<>();
-    private List<ComponentSearchResultDTO> connectionResults = new ArrayList<>();
-    private List<ComponentSearchResultDTO> processGroupResults = new ArrayList<>();
-    private List<ComponentSearchResultDTO> inputPortResults = new ArrayList<>();
-    private List<ComponentSearchResultDTO> outputPortResults = new ArrayList<>();
-    private List<ComponentSearchResultDTO> remoteProcessGroupResults = new ArrayList<>();
-    private List<ComponentSearchResultDTO> funnelResults = new ArrayList<>();
-
-    /**
-     * The processors that matched the search.
-     *
-     * @return
-     */
-    public List<ComponentSearchResultDTO> getProcessorResults() {
-        return processorResults;
-    }
-
-    public void setProcessorResults(List<ComponentSearchResultDTO> processorResults) {
-        this.processorResults = processorResults;
-    }
-
-    /**
-     * The connections that matched the search.
-     *
-     * @return
-     */
-    public List<ComponentSearchResultDTO> getConnectionResults() {
-        return connectionResults;
-    }
-
-    public void setConnectionResults(List<ComponentSearchResultDTO> connectionResults) {
-        this.connectionResults = connectionResults;
-    }
-
-    /**
-     * The process group that matched the search.
-     *
-     * @return
-     */
-    public List<ComponentSearchResultDTO> getProcessGroupResults() {
-        return processGroupResults;
-    }
-
-    public void setProcessGroupResults(List<ComponentSearchResultDTO> processGroupResults) {
-        this.processGroupResults = processGroupResults;
-    }
-
-    /**
-     * The input ports that matched the search.
-     *
-     * @return
-     */
-    public List<ComponentSearchResultDTO> getInputPortResults() {
-        return inputPortResults;
-    }
-
-    /**
-     * The output ports that matched the search.
-     *
-     * @return
-     */
-    public List<ComponentSearchResultDTO> getOutputPortResults() {
-        return outputPortResults;
-    }
-
-    public void setInputPortResults(List<ComponentSearchResultDTO> inputPortResults) {
-        this.inputPortResults = inputPortResults;
-    }
-
-    public void setOutputPortResults(List<ComponentSearchResultDTO> outputPortResults) {
-        this.outputPortResults = outputPortResults;
-    }
-
-    /**
-     * The remote process groups that matched the search.
-     *
-     * @return
-     */
-    public List<ComponentSearchResultDTO> getRemoteProcessGroupResults() {
-        return remoteProcessGroupResults;
-    }
-
-    public void setRemoteProcessGroupResults(List<ComponentSearchResultDTO> remoteProcessGroupResults) {
-        this.remoteProcessGroupResults = remoteProcessGroupResults;
-    }
-
-    /**
-     * The funnels that matched the search.
-     *
-     * @return
-     */
-    public List<ComponentSearchResultDTO> getFunnelResults() {
-        return funnelResults;
-    }
-
-    public void setFunnelResults(List<ComponentSearchResultDTO> funnelResults) {
-        this.funnelResults = funnelResults;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/UserGroupSearchResultDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/UserGroupSearchResultDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/UserGroupSearchResultDTO.java
deleted file mode 100644
index ab5636d..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/UserGroupSearchResultDTO.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.web.api.dto.search;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * A search result for a matching user group.
- */
-@XmlType(name = "userGroupSearchResult")
-public class UserGroupSearchResultDTO {
-
-    private String group;
-
-    /**
-     * The name of the group that matched.
-     *
-     * @return
-     */
-    public String getGroup() {
-        return group;
-    }
-
-    public void setGroup(String group) {
-        this.group = group;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/UserSearchResultDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/UserSearchResultDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/UserSearchResultDTO.java
deleted file mode 100644
index 9402eeb..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/UserSearchResultDTO.java
+++ /dev/null
@@ -1,56 +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.api.dto.search;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * A search result for a matching user.
- */
-@XmlType(name = "userSearchResult")
-public class UserSearchResultDTO {
-
-    private String userName;
-    private String userDn;
-
-    /**
-     * The dn of the user that matched.
-     *
-     * @return
-     */
-    public String getUserDn() {
-        return userDn;
-    }
-
-    public void setUserDn(String userDn) {
-        this.userDn = userDn;
-    }
-
-    /**
-     * The username of user that matched.
-     *
-     * @return
-     */
-    public String getUserName() {
-        return userName;
-    }
-
-    public void setUserName(String userName) {
-        this.userName = userName;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterConnectionStatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterConnectionStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterConnectionStatusDTO.java
deleted file mode 100644
index f0e11d6..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterConnectionStatusDTO.java
+++ /dev/null
@@ -1,89 +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.api.dto.status;
-
-import java.util.Collection;
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * DTO for serializing the a connection's status across the cluster.
- */
-@XmlType(name = "clusterConnectionStatus")
-public class ClusterConnectionStatusDTO {
-
-    private Collection<NodeConnectionStatusDTO> nodeConnectionStatus;
-    private Date statsLastRefreshed;
-    private String connectionId;
-    private String connectionName;
-
-    /**
-     * The time the status were last refreshed.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    public Date getStatsLastRefreshed() {
-        return statsLastRefreshed;
-    }
-
-    public void setStatsLastRefreshed(Date statsLastRefreshed) {
-        this.statsLastRefreshed = statsLastRefreshed;
-    }
-
-    /**
-     * The connection id.
-     *
-     * @return
-     */
-    public String getConnectionId() {
-        return connectionId;
-    }
-
-    public void setConnectionId(String connectionId) {
-        this.connectionId = connectionId;
-    }
-
-    /**
-     * The connection name.
-     *
-     * @return
-     */
-    public String getConnectionName() {
-        return connectionName;
-    }
-
-    public void setConnectionName(String connectionName) {
-        this.connectionName = connectionName;
-    }
-
-    /**
-     * Collection of node connection status DTO.
-     *
-     * @return The collection of node connection status DTO
-     */
-    public Collection<NodeConnectionStatusDTO> getNodeConnectionStatus() {
-        return nodeConnectionStatus;
-    }
-
-    public void setNodeConnectionStatus(Collection<NodeConnectionStatusDTO> nodeConnectionStatus) {
-        this.nodeConnectionStatus = nodeConnectionStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterPortStatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterPortStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterPortStatusDTO.java
deleted file mode 100644
index 4d7fa4d..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterPortStatusDTO.java
+++ /dev/null
@@ -1,89 +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.api.dto.status;
-
-import java.util.Collection;
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * DTO for serializing the a port's status across the cluster.
- */
-@XmlType(name = "clusterPortStatus")
-public class ClusterPortStatusDTO {
-
-    private Collection<NodePortStatusDTO> nodePortStatus;
-    private Date statsLastRefreshed;
-    private String portId;
-    private String portName;
-
-    /**
-     * The time the status was last refreshed.
-     *
-     * @return the time the status were last refreshed
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    public Date getStatsLastRefreshed() {
-        return statsLastRefreshed;
-    }
-
-    public void setStatsLastRefreshed(Date statsLastRefreshed) {
-        this.statsLastRefreshed = statsLastRefreshed;
-    }
-
-    /**
-     * The port status from each node in the cluster.
-     *
-     * @return
-     */
-    public Collection<NodePortStatusDTO> getNodePortStatus() {
-        return nodePortStatus;
-    }
-
-    public void setNodePortStatus(Collection<NodePortStatusDTO> nodePortStatus) {
-        this.nodePortStatus = nodePortStatus;
-    }
-
-    /**
-     * The port id.
-     *
-     * @return
-     */
-    public String getPortId() {
-        return portId;
-    }
-
-    public void setPortId(String portId) {
-        this.portId = portId;
-    }
-
-    /**
-     * The port name.
-     *
-     * @return
-     */
-    public String getPortName() {
-        return portName;
-    }
-
-    public void setPortName(String portName) {
-        this.portName = portName;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterProcessorStatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterProcessorStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterProcessorStatusDTO.java
deleted file mode 100644
index b9f45f2..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterProcessorStatusDTO.java
+++ /dev/null
@@ -1,117 +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.api.dto.status;
-
-import java.util.Collection;
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * DTO for serializing the a processor's status across the cluster.
- */
-@XmlType(name = "clusterProcessorStatus")
-public class ClusterProcessorStatusDTO {
-
-    private Collection<NodeProcessorStatusDTO> nodeProcessorStatus;
-    private Date statsLastRefreshed;
-    private String processorId;
-    private String processorName;
-    private String processorType;
-    private String processorRunStatus;
-
-    /**
-     * The time the status were last refreshed.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    public Date getStatsLastRefreshed() {
-        return statsLastRefreshed;
-    }
-
-    public void setStatsLastRefreshed(Date statsLastRefreshed) {
-        this.statsLastRefreshed = statsLastRefreshed;
-    }
-
-    /**
-     * The processor id.
-     *
-     * @return
-     */
-    public String getProcessorId() {
-        return processorId;
-    }
-
-    public void setProcessorId(String processorId) {
-        this.processorId = processorId;
-    }
-
-    /**
-     * The processor name.
-     *
-     * @return
-     */
-    public String getProcessorName() {
-        return processorName;
-    }
-
-    public void setProcessorName(String processorName) {
-        this.processorName = processorName;
-    }
-
-    /**
-     * The processor type.
-     *
-     * @return
-     */
-    public String getProcessorType() {
-        return processorType;
-    }
-
-    public void setProcessorType(String processorType) {
-        this.processorType = processorType;
-    }
-
-    /**
-     * The processor run status.
-     *
-     * @return
-     */
-    public String getProcessorRunStatus() {
-        return processorRunStatus;
-    }
-
-    public void setProcessorRunStatus(String runStatus) {
-        this.processorRunStatus = runStatus;
-    }
-
-    /**
-     * Collection of node processor status DTO.
-     *
-     * @return The collection of node processor status DTO
-     */
-    public Collection<NodeProcessorStatusDTO> getNodeProcessorStatus() {
-        return nodeProcessorStatus;
-    }
-
-    public void setNodeProcessorStatus(Collection<NodeProcessorStatusDTO> nodeProcessorStatus) {
-        this.nodeProcessorStatus = nodeProcessorStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterRemoteProcessGroupStatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterRemoteProcessGroupStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterRemoteProcessGroupStatusDTO.java
deleted file mode 100644
index d6b26d4..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterRemoteProcessGroupStatusDTO.java
+++ /dev/null
@@ -1,89 +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.api.dto.status;
-
-import java.util.Collection;
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * DTO for serializing the a remote process group's status across the cluster.
- */
-@XmlType(name = "clusterRemoteProcessGroupStatus")
-public class ClusterRemoteProcessGroupStatusDTO {
-
-    private Collection<NodeRemoteProcessGroupStatusDTO> nodeRemoteProcessGroupStatus;
-    private Date statsLastRefreshed;
-    private String remoteProcessGroupId;
-    private String remoteProcessGroupName;
-
-    /**
-     * The time the status was last refreshed.
-     *
-     * @return the time the status were last refreshed
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    public Date getStatsLastRefreshed() {
-        return statsLastRefreshed;
-    }
-
-    public void setStatsLastRefreshed(Date statsLastRefreshed) {
-        this.statsLastRefreshed = statsLastRefreshed;
-    }
-
-    /**
-     * The remote process group status from each node in the cluster.
-     *
-     * @return
-     */
-    public Collection<NodeRemoteProcessGroupStatusDTO> getNodeRemoteProcessGroupStatus() {
-        return nodeRemoteProcessGroupStatus;
-    }
-
-    public void setNodeRemoteProcessGroupStatus(Collection<NodeRemoteProcessGroupStatusDTO> nodeRemoteProcessGroupStatus) {
-        this.nodeRemoteProcessGroupStatus = nodeRemoteProcessGroupStatus;
-    }
-
-    /**
-     * The remote process group id.
-     *
-     * @return
-     */
-    public String getRemoteProcessGroupId() {
-        return remoteProcessGroupId;
-    }
-
-    public void setRemoteProcessGroupId(String remoteProcessGroupId) {
-        this.remoteProcessGroupId = remoteProcessGroupId;
-    }
-
-    /**
-     * The remote process group name.
-     *
-     * @return
-     */
-    public String getRemoteProcessGroupName() {
-        return remoteProcessGroupName;
-    }
-
-    public void setRemoteProcessGroupName(String remoteProcessGroupName) {
-        this.remoteProcessGroupName = remoteProcessGroupName;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusDTO.java
deleted file mode 100644
index d833991..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusDTO.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.web.api.dto.status;
-
-import java.util.Collection;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * DTO for serializing the controller status.
- */
-@XmlType(name = "clusterStatus")
-public class ClusterStatusDTO {
-
-    private Collection<NodeStatusDTO> nodeStatus;
-
-    /**
-     * The collection of the node status DTOs.
-     *
-     * @return
-     */
-    public Collection<NodeStatusDTO> getNodeStatus() {
-        return nodeStatus;
-    }
-
-    public void setNodeStatus(Collection<NodeStatusDTO> nodeStatus) {
-        this.nodeStatus = nodeStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusHistoryDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusHistoryDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusHistoryDTO.java
deleted file mode 100644
index 31a9948..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusHistoryDTO.java
+++ /dev/null
@@ -1,75 +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.api.dto.status;
-
-import java.util.Collection;
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * DTO for serializing the a status history across the cluster.
- */
-@XmlType(name = "clusterStatusHistory")
-public class ClusterStatusHistoryDTO {
-
-    private Collection<NodeStatusHistoryDTO> nodeStatusHistory;
-    private StatusHistoryDTO clusterStatusHistory;
-    private Date generated;
-
-    /**
-     * When this status history was generated.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    public Date getGenerated() {
-        return generated;
-    }
-
-    public void setGenerated(Date generated) {
-        this.generated = generated;
-    }
-
-    /**
-     * The status history from each node in the cluster.
-     *
-     * @return
-     */
-    public Collection<NodeStatusHistoryDTO> getNodeStatusHistory() {
-        return nodeStatusHistory;
-    }
-
-    public void setNodeStatusHistory(Collection<NodeStatusHistoryDTO> nodeStatusHistory) {
-        this.nodeStatusHistory = nodeStatusHistory;
-    }
-
-    /**
-     * The status history for this component across the entire cluster.
-     *
-     * @return
-     */
-    public StatusHistoryDTO getClusterStatusHistory() {
-        return clusterStatusHistory;
-    }
-
-    public void setClusterStatusHistory(StatusHistoryDTO clusterStatusHistory) {
-        this.clusterStatusHistory = clusterStatusHistory;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusDTO.java
deleted file mode 100644
index 387cce5..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusDTO.java
+++ /dev/null
@@ -1,198 +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.api.dto.status;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * DTO for serializing the status of a connection.
- */
-@XmlType(name = "connectionStatus")
-public class ConnectionStatusDTO {
-
-    private String id;
-    private String groupId;
-    private String name;
-    private String input;
-    private String queuedCount;
-    private String queuedSize;
-    private String queued;
-    private String output;
-
-    private String sourceId;
-    private String sourceName;
-    private String destinationId;
-    private String destinationName;
-
-    /* getters / setters */
-    /**
-     * The id for the connection.
-     *
-     * @return The connection id
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The ID of the Process Group to which this processor belongs.
-     *
-     * @return the ID of the Process Group to which this processor belongs.
-     */
-    public String getGroupId() {
-        return groupId;
-    }
-
-    public void setGroupId(final String groupId) {
-        this.groupId = groupId;
-    }
-
-    /**
-     * The name of this connection.
-     *
-     * @return
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * The total count of flow files that are queued.
-     *
-     * @return
-     */
-    public String getQueuedCount() {
-        return queuedCount;
-    }
-
-    public void setQueuedCount(String queuedCount) {
-        this.queuedCount = queuedCount;
-    }
-
-    /**
-     * The total size of flow files that are queued.
-     *
-     * @return
-     */
-    public String getQueuedSize() {
-        return queuedSize;
-    }
-
-    public void setQueuedSize(String queuedSize) {
-        this.queuedSize = queuedSize;
-    }
-
-    /**
-     * The total count and size of flow files that are queued.
-     *
-     * @return The total count and size of queued flow files
-     */
-    public String getQueued() {
-        return queued;
-    }
-
-    public void setQueued(String queued) {
-        this.queued = queued;
-    }
-
-    /**
-     * The id of the source of this connection.
-     *
-     * @return
-     */
-    public String getSourceId() {
-        return sourceId;
-    }
-
-    public void setSourceId(String sourceId) {
-        this.sourceId = sourceId;
-    }
-
-    /**
-     * The name of the source of this connection.
-     *
-     * @return
-     */
-    public String getSourceName() {
-        return sourceName;
-    }
-
-    public void setSourceName(String sourceName) {
-        this.sourceName = sourceName;
-    }
-
-    /**
-     * The id of the destination of this connection.
-     *
-     * @return
-     */
-    public String getDestinationId() {
-        return destinationId;
-    }
-
-    public void setDestinationId(String destinationId) {
-        this.destinationId = destinationId;
-    }
-
-    /**
-     * The name of the destination of this connection.
-     *
-     * @return
-     */
-    public String getDestinationName() {
-        return destinationName;
-    }
-
-    public void setDestinationName(String destinationName) {
-        this.destinationName = destinationName;
-    }
-
-    /**
-     * The input for this connection.
-     *
-     * @return
-     */
-    public String getInput() {
-        return input;
-    }
-
-    public void setInput(String input) {
-        this.input = input;
-    }
-
-    /**
-     * The output for this connection.
-     *
-     * @return
-     */
-    public String getOutput() {
-        return output;
-    }
-
-    public void setOutput(String output) {
-        this.output = output;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.java
deleted file mode 100644
index 9e063e4..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.java
+++ /dev/null
@@ -1,187 +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.api.dto.status;
-
-import java.util.List;
-import javax.xml.bind.annotation.XmlType;
-import org.apache.nifi.web.api.dto.BulletinDTO;
-
-/**
- * The status of this NiFi controller.
- */
-@XmlType(name = "controllerStatus")
-public class ControllerStatusDTO {
-
-    private Integer activeThreadCount;
-    private String queued;
-    private String connectedNodes;
-    private Boolean hasPendingAccounts;
-
-    private Integer runningCount;
-    private Integer stoppedCount;
-    private Integer invalidCount;
-    private Integer disabledCount;
-    private Integer activeRemotePortCount;
-    private Integer inactiveRemotePortCount;
-
-    private List<BulletinDTO> bulletins;
-
-    /**
-     * The active thread count.
-     *
-     * @return The active thread count
-     */
-    public Integer getActiveThreadCount() {
-        return activeThreadCount;
-    }
-
-    public void setActiveThreadCount(Integer activeThreadCount) {
-        this.activeThreadCount = activeThreadCount;
-    }
-
-    /**
-     * The queue for the controller.
-     *
-     * @return
-     */
-    public String getQueued() {
-        return queued;
-    }
-
-    public void setQueued(String queued) {
-        this.queued = queued;
-    }
-
-    /**
-     * Used in clustering, will report the number of nodes connected vs the
-     * number of nodes in the cluster.
-     *
-     * @return
-     */
-    public String getConnectedNodes() {
-        return connectedNodes;
-    }
-
-    public void setConnectedNodes(String connectedNodes) {
-        this.connectedNodes = connectedNodes;
-    }
-
-    /**
-     * System bulletins to be reported to the user.
-     *
-     * @return
-     */
-    public List<BulletinDTO> getBulletins() {
-        return bulletins;
-    }
-
-    public void setBulletins(List<BulletinDTO> bulletins) {
-        this.bulletins = bulletins;
-    }
-
-    /**
-     * Whether or not there are pending user requests.
-     *
-     * @return
-     */
-    public Boolean getHasPendingAccounts() {
-        return hasPendingAccounts;
-    }
-
-    public void setHasPendingAccounts(Boolean hasPendingAccounts) {
-        this.hasPendingAccounts = hasPendingAccounts;
-    }
-
-    /**
-     * The number of running components in this controller.
-     *
-     * @return
-     */
-    public Integer getRunningCount() {
-        return runningCount;
-    }
-
-    public void setRunningCount(Integer runningCount) {
-        this.runningCount = runningCount;
-    }
-
-    /**
-     * The number of stopped components in this controller.
-     *
-     * @return
-     */
-    public Integer getStoppedCount() {
-        return stoppedCount;
-    }
-
-    public void setStoppedCount(Integer stoppedCount) {
-        this.stoppedCount = stoppedCount;
-    }
-
-    /**
-     * The number of invalid components in this controller.
-     *
-     * @return
-     */
-    public Integer getInvalidCount() {
-        return invalidCount;
-    }
-
-    public void setInvalidCount(Integer invalidCount) {
-        this.invalidCount = invalidCount;
-    }
-
-    /**
-     * The number of disabled components in this controller.
-     *
-     * @return
-     */
-    public Integer getDisabledCount() {
-        return disabledCount;
-    }
-
-    public void setDisabledCount(Integer disabledCount) {
-        this.disabledCount = disabledCount;
-    }
-
-    /**
-     * The number of active remote ports in this controller.
-     *
-     * @return
-     */
-    public Integer getActiveRemotePortCount() {
-        return activeRemotePortCount;
-    }
-
-    public void setActiveRemotePortCount(Integer activeRemotePortCount) {
-        this.activeRemotePortCount = activeRemotePortCount;
-    }
-
-    /**
-     * The number of inactive remote ports in this controller.
-     *
-     * @return
-     */
-    public Integer getInactiveRemotePortCount() {
-        return inactiveRemotePortCount;
-    }
-
-    public void setInactiveRemotePortCount(Integer inactiveRemotePortCount) {
-        this.inactiveRemotePortCount = inactiveRemotePortCount;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusDTO.java
deleted file mode 100644
index f7cba44..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusDTO.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.api.dto.status;
-
-import javax.xml.bind.annotation.XmlType;
-import org.apache.nifi.web.api.dto.NodeDTO;
-
-/**
- * DTO for serializing the connection status for a particular node.
- */
-@XmlType(name = "nodeConnectionStatus")
-public class NodeConnectionStatusDTO {
-
-    private NodeDTO node;
-    private ConnectionStatusDTO connectionStatus;
-
-    /**
-     * The node.
-     *
-     * @return
-     */
-    public NodeDTO getNode() {
-        return node;
-    }
-
-    public void setNode(NodeDTO node) {
-        this.node = node;
-    }
-
-    /**
-     * The connection's status.
-     *
-     * @return
-     */
-    public ConnectionStatusDTO getConnectionStatus() {
-        return connectionStatus;
-    }
-
-    public void setConnectionStatus(ConnectionStatusDTO connectionStatus) {
-        this.connectionStatus = connectionStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodePortStatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodePortStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodePortStatusDTO.java
deleted file mode 100644
index 93d89ac..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodePortStatusDTO.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.api.dto.status;
-
-import javax.xml.bind.annotation.XmlType;
-import org.apache.nifi.web.api.dto.NodeDTO;
-
-/**
- * DTO for serializing the port status for a particular node.
- */
-@XmlType(name = "nodePortStatus")
-public class NodePortStatusDTO {
-
-    private NodeDTO node;
-    private PortStatusDTO portStatus;
-
-    /**
-     * The node.
-     *
-     * @return
-     */
-    public NodeDTO getNode() {
-        return node;
-    }
-
-    public void setNode(NodeDTO node) {
-        this.node = node;
-    }
-
-    /**
-     * The port's status.
-     *
-     * @return
-     */
-    public PortStatusDTO getPortStatus() {
-        return portStatus;
-    }
-
-    public void setPortStatus(PortStatusDTO portStatus) {
-        this.portStatus = portStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusDTO.java
deleted file mode 100644
index afff4b1..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusDTO.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.api.dto.status;
-
-import javax.xml.bind.annotation.XmlType;
-import org.apache.nifi.web.api.dto.NodeDTO;
-
-/**
- * DTO for serializing the processor status for a particular node.
- */
-@XmlType(name = "nodeProcessorStatus")
-public class NodeProcessorStatusDTO {
-
-    private NodeDTO node;
-    private ProcessorStatusDTO processorStatus;
-
-    /**
-     * The node.
-     *
-     * @return
-     */
-    public NodeDTO getNode() {
-        return node;
-    }
-
-    public void setNode(NodeDTO node) {
-        this.node = node;
-    }
-
-    /**
-     * The processor's status.
-     *
-     * @return
-     */
-    public ProcessorStatusDTO getProcessorStatus() {
-        return processorStatus;
-    }
-
-    public void setProcessorStatus(ProcessorStatusDTO processorStatus) {
-        this.processorStatus = processorStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusDTO.java
deleted file mode 100644
index 03923bf..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusDTO.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.api.dto.status;
-
-import javax.xml.bind.annotation.XmlType;
-import org.apache.nifi.web.api.dto.NodeDTO;
-
-/**
- * DTO for serializing the remote process group status for a particular node.
- */
-@XmlType(name = "nodeRemoteProcessGroupStatus")
-public class NodeRemoteProcessGroupStatusDTO {
-
-    private NodeDTO node;
-    private RemoteProcessGroupStatusDTO remoteProcessGroupStatus;
-
-    /**
-     * The node.
-     *
-     * @return
-     */
-    public NodeDTO getNode() {
-        return node;
-    }
-
-    public void setNode(NodeDTO node) {
-        this.node = node;
-    }
-
-    /**
-     * The remote process group's status.
-     *
-     * @return
-     */
-    public RemoteProcessGroupStatusDTO getRemoteProcessGroupStatus() {
-        return remoteProcessGroupStatus;
-    }
-
-    public void setRemoteProcessGroupStatus(RemoteProcessGroupStatusDTO remoteProcessGroupStatus) {
-        this.remoteProcessGroupStatus = remoteProcessGroupStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusDTO.java
deleted file mode 100644
index 03647cd..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusDTO.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.api.dto.status;
-
-import javax.xml.bind.annotation.XmlType;
-import org.apache.nifi.web.api.dto.NodeDTO;
-
-/**
- * DTO for serializing the node status.
- */
-@XmlType(name = "nodeStatus")
-public class NodeStatusDTO {
-
-    private NodeDTO node;
-    private ProcessGroupStatusDTO controllerStatus;
-
-    /**
-     * The node.
-     *
-     * @return
-     */
-    public NodeDTO getNode() {
-        return node;
-    }
-
-    public void setNode(NodeDTO node) {
-        this.node = node;
-    }
-
-    /**
-     * The controller status.
-     *
-     * @return
-     */
-    public ProcessGroupStatusDTO getControllerStatus() {
-        return controllerStatus;
-    }
-
-    public void setControllerStatus(ProcessGroupStatusDTO controllerStatus) {
-        this.controllerStatus = controllerStatus;
-    }
-
-}


[06/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/EventDrivenWorkerQueue.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/EventDrivenWorkerQueue.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/EventDrivenWorkerQueue.java
deleted file mode 100644
index f36a459..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/EventDrivenWorkerQueue.java
+++ /dev/null
@@ -1,329 +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;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.nifi.connectable.Connectable;
-import org.apache.nifi.connectable.Connection;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.util.Connectables;
-
-public class EventDrivenWorkerQueue implements WorkerQueue {
-
-    private final Object workMonitor = new Object();
-
-    private final Map<Connectable, Worker> workerMap = new HashMap<>();   // protected by synchronizing on workMonitor
-    private final WorkerReadyQueue workerQueue;
-
-    public EventDrivenWorkerQueue(final boolean clustered, final boolean primary, final ProcessScheduler scheduler) {
-        workerQueue = new WorkerReadyQueue(scheduler);
-        workerQueue.setClustered(clustered);
-        workerQueue.setPrimary(primary);
-    }
-
-    @Override
-    public void setClustered(final boolean clustered) {
-        workerQueue.setClustered(clustered);
-    }
-
-    @Override
-    public void setPrimary(final boolean primary) {
-        workerQueue.setPrimary(primary);
-    }
-
-    @Override
-    public Worker poll(final long timeout, final TimeUnit timeUnit) {
-        final long maxTime = System.currentTimeMillis() + TimeUnit.MILLISECONDS.convert(timeout, timeUnit);
-        while (System.currentTimeMillis() < maxTime) {
-            synchronized (workMonitor) {
-                final Worker worker = workerQueue.poll();
-                if (worker == null) {
-                    // nothing to do. wait until we have something to do.
-                    final long timeLeft = maxTime - System.currentTimeMillis();
-                    if (timeLeft <= 0) {
-                        return null;
-                    }
-
-                    try {
-                        workMonitor.wait(timeLeft);
-                    } catch (final InterruptedException ignored) {
-                    }
-                } else {
-                    // Decrement the amount of work there is to do for this worker.
-                    final int workLeft = worker.decrementEventCount();
-                    if (workLeft > 0) {
-                        workerQueue.offer(worker);
-                    }
-
-                    return worker;
-                }
-            }
-        }
-
-        return null;
-    }
-
-    @Override
-    public void offer(final Connectable connectable) {
-        synchronized (workMonitor) {
-            Worker worker = workerMap.get(connectable);
-            if (worker == null) {
-                // if worker is null, then it has not been scheduled to run; ignore the event.
-                return;
-            }
-
-            final int countBefore = worker.incrementEventCount();
-            if (countBefore < 0) {
-                worker.setWorkCount(1);
-            }
-            if (countBefore <= 0) {
-                // If countBefore > 0 then it's already on the queue, so just incrementing its counter is sufficient.
-                workerQueue.offer(worker);
-            }
-
-            workMonitor.notify();
-        }
-    }
-
-    private int getWorkCount(final Connectable connectable) {
-        int sum = 0;
-        for (final Connection connection : connectable.getIncomingConnections()) {
-            sum += connection.getFlowFileQueue().size().getObjectCount();
-        }
-        return sum;
-    }
-
-    @Override
-    public void resumeWork(final Connectable connectable) {
-        synchronized (workMonitor) {
-            final int workCount = getWorkCount(connectable);
-            final Worker worker = new Worker(connectable);
-            workerMap.put(connectable, worker);
-
-            if (workCount > 0) {
-                worker.setWorkCount(workCount);
-                workerQueue.offer(worker);
-                workMonitor.notify();
-            }
-        }
-    }
-
-    @Override
-    public void suspendWork(final Connectable connectable) {
-        synchronized (workMonitor) {
-            final Worker worker = this.workerMap.remove(connectable);
-            if (worker == null) {
-                return;
-            }
-
-            worker.resetWorkCount();
-            workerQueue.remove(worker);
-        }
-    }
-
-    public static class Worker implements EventBasedWorker {
-
-        private final Connectable connectable;
-        private final AtomicInteger workCount = new AtomicInteger(0);
-
-        public Worker(final Connectable connectable) {
-            this.connectable = connectable;
-        }
-
-        @Override
-        public Connectable getConnectable() {
-            return connectable;
-        }
-
-        @Override
-        public int decrementEventCount() {
-            return workCount.decrementAndGet();
-        }
-
-        @Override
-        public int incrementEventCount() {
-            return workCount.getAndIncrement();
-        }
-
-        void resetWorkCount() {
-            workCount.set(0);
-        }
-
-        void setWorkCount(final int workCount) {
-            this.workCount.set(workCount);
-        }
-    }
-
-    @SuppressWarnings("serial")
-    private static class WorkerReadyQueue extends LinkedList<Worker> {
-
-        private final ProcessScheduler scheduler;
-
-        private volatile boolean clustered = false;
-        private volatile boolean primary = false;
-
-        public WorkerReadyQueue(final ProcessScheduler scheduler) {
-            this.scheduler = scheduler;
-        }
-
-        public void setClustered(final boolean clustered) {
-            this.clustered = clustered;
-        }
-
-        public void setPrimary(final boolean primary) {
-            this.primary = primary;
-        }
-
-        @Override
-        public Worker poll() {
-            final List<Worker> putBack = new ArrayList<>();
-
-            Worker worker;
-            try {
-                while ((worker = super.poll()) != null) {
-                    final DelayProcessingReason reason = getDelayReason(worker);
-                    if (reason == null) {
-                        return worker;
-                    } else {
-                        // Worker is not ready. We may want to add him back to the queue, depending on the reason that he is unready.
-                        switch (reason) {
-                            case YIELDED:
-                            case ISOLATED:
-                            case DESTINATION_FULL:
-                            case ALL_WORK_PENALIZED:
-                            case NO_WORK:
-                            case TOO_MANY_THREADS:
-                                // there will not be an event that triggers this to happen, so we add this worker back to the queue.
-                                putBack.add(worker);
-                                break;
-                            default:
-                            case NOT_RUNNING:
-                                // There's no need to check if this worker is available again until a another event
-                                // occurs. Therefore, we keep him off of the queue and reset his work count
-                                worker.resetWorkCount();
-                                break;
-                        }
-                    }
-                }
-            } finally {
-                if (!putBack.isEmpty()) {
-                    super.addAll(putBack);
-                }
-            }
-
-            return null;
-        }
-
-        private DelayProcessingReason getDelayReason(final Worker worker) {
-            final Connectable connectable = worker.getConnectable();
-
-            if (ScheduledState.RUNNING != connectable.getScheduledState()) {
-                return DelayProcessingReason.NOT_RUNNING;
-            }
-
-            if (connectable.getYieldExpiration() > System.currentTimeMillis()) {
-                return DelayProcessingReason.YIELDED;
-            }
-
-            // For Remote Output Ports,
-            int availableRelationshipCount = 0;
-            if (!connectable.getRelationships().isEmpty()) {
-                availableRelationshipCount = getAvailableRelationshipCount(connectable);
-
-                if (availableRelationshipCount == 0) {
-                    return DelayProcessingReason.DESTINATION_FULL;
-                }
-            }
-
-            if (connectable.hasIncomingConnection() && !Connectables.flowFilesQueued(connectable)) {
-                return DelayProcessingReason.NO_WORK;
-            }
-
-            final int activeThreadCount = scheduler.getActiveThreadCount(worker.getConnectable());
-            final int maxThreadCount = worker.getConnectable().getMaxConcurrentTasks();
-            if (maxThreadCount > 0 && activeThreadCount >= maxThreadCount) {
-                return DelayProcessingReason.TOO_MANY_THREADS;
-            }
-
-            if (connectable instanceof ProcessorNode) {
-                final ProcessorNode procNode = (ProcessorNode) connectable;
-                if (procNode.isIsolated() && clustered && !primary) {
-                    return DelayProcessingReason.ISOLATED;
-                }
-
-                final boolean triggerWhenAnyAvailable = procNode.isTriggerWhenAnyDestinationAvailable();
-                final boolean allDestinationsAvailable = availableRelationshipCount == procNode.getRelationships().size();
-                if (!triggerWhenAnyAvailable && !allDestinationsAvailable) {
-                    return DelayProcessingReason.DESTINATION_FULL;
-                }
-            }
-
-            return null;
-        }
-
-        private int getAvailableRelationshipCount(final Connectable connectable) {
-            int count = 0;
-            for (final Relationship relationship : connectable.getRelationships()) {
-                final Collection<Connection> connections = connectable.getConnections(relationship);
-
-                if (connections == null || connections.isEmpty()) {
-                    if (connectable.isAutoTerminated(relationship)) {
-                        // If the relationship is auto-terminated, consider it available.
-                        count++;
-                    }
-                } else {
-                    boolean available = true;
-                    for (final Connection connection : connections) {
-                        if (connection.getSource() == connection.getDestination()) {
-                            // don't count self-loops
-                            continue;
-                        }
-
-                        if (connection.getFlowFileQueue().isFull()) {
-                            available = false;
-                        }
-                    }
-
-                    if (available) {
-                        count++;
-                    }
-                }
-            }
-
-            return count;
-        }
-    }
-
-    private static enum DelayProcessingReason {
-
-        YIELDED,
-        DESTINATION_FULL,
-        NO_WORK,
-        ALL_WORK_PENALIZED,
-        ISOLATED,
-        NOT_RUNNING,
-        TOO_MANY_THREADS;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java
deleted file mode 100644
index e1d80b0..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java
+++ /dev/null
@@ -1,768 +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;
-
-import java.io.BufferedInputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.UUID;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.regex.Pattern;
-
-import org.apache.nifi.controller.repository.ConnectionSwapInfo;
-import org.apache.nifi.controller.repository.FlowFileRecord;
-import org.apache.nifi.controller.repository.FlowFileRepository;
-import org.apache.nifi.controller.repository.FlowFileSwapManager;
-import org.apache.nifi.controller.repository.QueueProvider;
-import org.apache.nifi.controller.repository.StandardFlowFileRecord;
-import org.apache.nifi.controller.repository.claim.ContentClaim;
-import org.apache.nifi.controller.repository.claim.ContentClaimManager;
-import org.apache.nifi.engine.FlowEngine;
-import org.apache.nifi.events.EventReporter;
-import org.apache.nifi.stream.io.BufferedOutputStream;
-import org.apache.nifi.processor.QueueSize;
-import org.apache.nifi.reporting.Severity;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.NiFiProperties;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * <p>
- * An implementation of the {@link FlowFileSwapManager} that swaps FlowFiles
- * to/from local disk
- * </p>
- */
-public class FileSystemSwapManager implements FlowFileSwapManager {
-
-    public static final int MINIMUM_SWAP_COUNT = 10000;
-    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 = 6;
-    public static final String EVENT_CATEGORY = "Swap FlowFiles";
-
-    private final ScheduledExecutorService swapQueueIdentifierExecutor;
-    private final ScheduledExecutorService swapInExecutor;
-    private volatile FlowFileRepository flowFileRepository;
-    private volatile EventReporter eventReporter;
-
-    // Maintains a mapping of FlowFile Queue to the a QueueLockWrapper, which provides queue locking and necessary state for swapping back in
-    private final ConcurrentMap<FlowFileQueue, QueueLockWrapper> swapMap = new ConcurrentHashMap<>();
-    private final File storageDirectory;
-    private final long swapInMillis;
-    private final long swapOutMillis;
-    private final int swapOutThreadCount;
-
-    private ContentClaimManager claimManager;	// effectively final
-
-    private static final Logger logger = LoggerFactory.getLogger(FileSystemSwapManager.class);
-
-    public FileSystemSwapManager() {
-        final NiFiProperties properties = NiFiProperties.getInstance();
-        final Path flowFileRepoPath = properties.getFlowFileRepositoryPath();
-        
-        this.storageDirectory = flowFileRepoPath.resolve("swap").toFile();
-        if (!storageDirectory.exists() && !storageDirectory.mkdirs()) {
-            throw new RuntimeException("Cannot create Swap Storage directory " + storageDirectory.getAbsolutePath());
-        }
-
-        swapQueueIdentifierExecutor = new FlowEngine(1, "Identifies Queues for FlowFile Swapping");
-
-        swapInMillis = FormatUtils.getTimeDuration(properties.getSwapInPeriod(), TimeUnit.MILLISECONDS);
-        swapOutMillis = FormatUtils.getTimeDuration(properties.getSwapOutPeriod(), TimeUnit.MILLISECONDS);
-        swapOutThreadCount = properties.getSwapOutThreads();
-        swapInExecutor = new FlowEngine(properties.getSwapInThreads(), "Swap In FlowFiles");
-    }
-
-    @Override
-    public void purge() {
-        final File[] swapFiles = storageDirectory.listFiles(new FilenameFilter() {
-            @Override
-            public boolean accept(final File dir, final String name) {
-                return SWAP_FILE_PATTERN.matcher(name).matches();
-            }
-        });
-
-        if (swapFiles != null) {
-            for (final File file : swapFiles) {
-                if (!file.delete() && file.exists()) {
-                    logger.warn("Failed to delete SWAP file {}", file);
-                }
-            }
-        }
-    }
-
-    public synchronized void start(final FlowFileRepository flowFileRepository, final QueueProvider connectionProvider, final ContentClaimManager claimManager, final EventReporter eventReporter) {
-        this.claimManager = claimManager;
-        this.flowFileRepository = flowFileRepository;
-        this.eventReporter = eventReporter;
-        swapQueueIdentifierExecutor.scheduleWithFixedDelay(new QueueIdentifier(connectionProvider), swapOutMillis, swapOutMillis, TimeUnit.MILLISECONDS);
-        swapInExecutor.scheduleWithFixedDelay(new SwapInTask(), swapInMillis, swapInMillis, TimeUnit.MILLISECONDS);
-    }
-
-    public int serializeFlowFiles(final List<FlowFileRecord> toSwap, final FlowFileQueue queue, final String swapLocation, final OutputStream destination) throws IOException {
-        if (toSwap == null || toSwap.isEmpty()) {
-            return 0;
-        }
-
-        long contentSize = 0L;
-        for (final FlowFileRecord record : toSwap) {
-            contentSize += record.getSize();
-        }
-
-        // persist record to disk via the swap file
-        final OutputStream bufferedOut = new BufferedOutputStream(destination);
-        final DataOutputStream out = new DataOutputStream(bufferedOut);
-        try {
-            out.writeInt(SWAP_ENCODING_VERSION);
-            out.writeUTF(queue.getIdentifier());
-            out.writeInt(toSwap.size());
-            out.writeLong(contentSize);
-
-            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.getLastQueueDate());
-                out.writeLong(flowFile.getSize());
-
-                final ContentClaim claim = flowFile.getContentClaim();
-                if (claim == null) {
-                    out.writeBoolean(false);
-                } else {
-                    out.writeBoolean(true);
-                    out.writeUTF(claim.getId());
-                    out.writeUTF(claim.getContainer());
-                    out.writeUTF(claim.getSection());
-                    out.writeLong(flowFile.getContentClaimOffset());
-                    out.writeBoolean(claim.isLossTolerant());
-                }
-
-                final Map<String, String> attributes = flowFile.getAttributes();
-                out.writeInt(attributes.size());
-                for (final Map.Entry<String, String> entry : attributes.entrySet()) {
-                    writeString(entry.getKey(), out);
-                    writeString(entry.getValue(), out);
-                }
-            }
-        } finally {
-            out.flush();
-        }
-
-        logger.info("Successfully swapped out {} FlowFiles from {} to Swap File {}", new Object[]{toSwap.size(), queue, swapLocation});
-
-        return toSwap.size();
-    }
-
-    private void writeString(final String toWrite, final OutputStream out) throws IOException {
-        final byte[] bytes = toWrite.getBytes("UTF-8");
-        final int utflen = bytes.length;
-
-        if (utflen < 65535) {
-            out.write(utflen >>> 8);
-            out.write(utflen);
-            out.write(bytes);
-        } else {
-            out.write(255);
-            out.write(255);
-            out.write(utflen >>> 24);
-            out.write(utflen >>> 16);
-            out.write(utflen >>> 8);
-            out.write(utflen);
-            out.write(bytes);
-        }
-    }
-
-    static List<FlowFileRecord> deserializeFlowFiles(final DataInputStream in, final FlowFileQueue queue, final ContentClaimManager claimManager) throws IOException {
-        final int swapEncodingVersion = in.readInt();
-        if (swapEncodingVersion > SWAP_ENCODING_VERSION) {
-            throw new IOException("Cannot swap FlowFiles in from SwapFile because the encoding version is "
-                    + swapEncodingVersion + ", which is too new (expecting " + SWAP_ENCODING_VERSION + " or less)");
-        }
-
-        final String connectionId = in.readUTF();
-        if (!connectionId.equals(queue.getIdentifier())) {
-            throw new IllegalArgumentException("Cannot restore Swap File because the file indicates that records belong to Connection with ID " + connectionId + " but received Connection " + queue);
-        }
-
-        final int numRecords = in.readInt();
-        in.readLong();  // Content Size
-
-        return deserializeFlowFiles(in, numRecords, queue, swapEncodingVersion, false, claimManager);
-    }
-
-    static List<FlowFileRecord> deserializeFlowFiles(final DataInputStream in, final int numFlowFiles, final FlowFileQueue queue, final int serializationVersion, final boolean incrementContentClaims, final ContentClaimManager claimManager) throws IOException {
-        final List<FlowFileRecord> flowFiles = new ArrayList<>();
-        for (int i = 0; i < numFlowFiles; i++) {
-            // legacy encoding had an "action" because it used to be couple with FlowFile Repository code
-            if (serializationVersion < 3) {
-                final int action = in.read();
-                if (action != 1) {
-                    throw new IOException("Swap File is version " + serializationVersion + " but did not contain a 'UPDATE' record type");
-                }
-            }
-
-            final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder();
-            ffBuilder.id(in.readLong());
-            ffBuilder.entryDate(in.readLong());
-
-            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());
-                }
-                ffBuilder.lineageIdentifiers(lineageIdentifiers);
-                ffBuilder.lineageStartDate(in.readLong());
-
-                if (serializationVersion > 5) {
-                    ffBuilder.lastQueueDate(in.readLong());
-                }
-            }
-
-            ffBuilder.size(in.readLong());
-
-            if (serializationVersion < 3) {
-                readString(in); // connection Id
-            }
-
-            final boolean hasClaim = in.readBoolean();
-            if (hasClaim) {
-                final String claimId;
-                if (serializationVersion < 5) {
-                    claimId = String.valueOf(in.readLong());
-                } else {
-                    claimId = in.readUTF();
-                }
-
-                final String container = in.readUTF();
-                final String section = in.readUTF();
-                final long claimOffset = in.readLong();
-
-                final boolean lossTolerant;
-                if (serializationVersion >= 4) {
-                    lossTolerant = in.readBoolean();
-                } else {
-                    lossTolerant = false;
-                }
-
-                final ContentClaim claim = claimManager.newContentClaim(container, section, claimId, lossTolerant);
-
-                if (incrementContentClaims) {
-                    claimManager.incrementClaimantCount(claim);
-                }
-
-                ffBuilder.contentClaim(claim);
-                ffBuilder.contentClaimOffset(claimOffset);
-            }
-
-            boolean attributesChanged = true;
-            if (serializationVersion < 3) {
-                attributesChanged = in.readBoolean();
-            }
-
-            if (attributesChanged) {
-                final int numAttributes = in.readInt();
-                for (int j = 0; j < numAttributes; j++) {
-                    final String key = readString(in);
-                    final String value = readString(in);
-
-                    ffBuilder.addAttribute(key, value);
-                }
-            }
-
-            final FlowFileRecord record = ffBuilder.build();
-            flowFiles.add(record);
-        }
-
-        return flowFiles;
-    }
-
-    private static String readString(final InputStream in) throws IOException {
-        final Integer numBytes = readFieldLength(in);
-        if (numBytes == null) {
-            throw new EOFException();
-        }
-        final byte[] bytes = new byte[numBytes];
-        fillBuffer(in, bytes, numBytes);
-        return new String(bytes, "UTF-8");
-    }
-
-    private static Integer readFieldLength(final InputStream in) throws IOException {
-        final int firstValue = in.read();
-        final int secondValue = in.read();
-        if (firstValue < 0) {
-            return null;
-        }
-        if (secondValue < 0) {
-            throw new EOFException();
-        }
-        if (firstValue == 0xff && secondValue == 0xff) {
-            int ch1 = in.read();
-            int ch2 = in.read();
-            int ch3 = in.read();
-            int ch4 = in.read();
-            if ((ch1 | ch2 | ch3 | ch4) < 0) {
-                throw new EOFException();
-            }
-            return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4));
-        } else {
-            return ((firstValue << 8) + (secondValue));
-        }
-    }
-
-    private static void fillBuffer(final InputStream in, final byte[] buffer, final int length) throws IOException {
-        int bytesRead;
-        int totalBytesRead = 0;
-        while ((bytesRead = in.read(buffer, totalBytesRead, length - totalBytesRead)) > 0) {
-            totalBytesRead += bytesRead;
-        }
-        if (totalBytesRead != length) {
-            throw new EOFException();
-        }
-    }
-
-    private class QueueIdentifier implements Runnable {
-
-        private final QueueProvider connectionProvider;
-
-        public QueueIdentifier(final QueueProvider connectionProvider) {
-            this.connectionProvider = connectionProvider;
-        }
-
-        @Override
-        public void run() {
-            final Collection<FlowFileQueue> allQueues = connectionProvider.getAllQueues();
-            final BlockingQueue<FlowFileQueue> connectionQueue = new LinkedBlockingQueue<>(allQueues);
-
-            final ThreadFactory threadFactory = new ThreadFactory() {
-                @Override
-                public Thread newThread(final Runnable r) {
-                    final Thread t = new Thread(r);
-                    t.setName("Swap Out FlowFiles");
-                    return t;
-                }
-            };
-
-            final ExecutorService workerExecutor = Executors.newFixedThreadPool(swapOutThreadCount, threadFactory);
-            for (int i = 0; i < swapOutThreadCount; i++) {
-                workerExecutor.submit(new SwapOutTask(connectionQueue));
-            }
-
-            workerExecutor.shutdown();
-
-            try {
-                workerExecutor.awaitTermination(10, TimeUnit.MINUTES);
-            } catch (final InterruptedException e) {
-                // oh well...
-            }
-        }
-    }
-
-    private class SwapInTask implements Runnable {
-
-        @Override
-        public void run() {
-            for (final Map.Entry<FlowFileQueue, QueueLockWrapper> entry : swapMap.entrySet()) {
-                final FlowFileQueue flowFileQueue = entry.getKey();
-
-                // if queue is more than 60% of its swap threshold, don't swap flowfiles in
-                if (flowFileQueue.unswappedSize() >= ((float) flowFileQueue.getSwapThreshold() * 0.6F)) {
-                    continue;
-                }
-
-                final QueueLockWrapper queueLockWrapper = entry.getValue();
-                if (queueLockWrapper.getLock().tryLock()) {
-                    try {
-                        final Queue<File> queue = queueLockWrapper.getQueue();
-
-                        // Swap FlowFiles in until we hit 90% of the threshold, or until we're out of files.
-                        while (flowFileQueue.unswappedSize() < ((float) flowFileQueue.getSwapThreshold() * 0.9F)) {
-                            File swapFile = null;
-                            try {
-                                swapFile = queue.poll();
-                                if (swapFile == null) {
-                                    break;
-                                }
-
-                                try (final InputStream fis = new FileInputStream(swapFile);
-                                        final DataInputStream in = new DataInputStream(fis)) {
-                                    final List<FlowFileRecord> swappedFlowFiles = deserializeFlowFiles(in, flowFileQueue, claimManager);
-                                    flowFileRepository.swapFlowFilesIn(swapFile.getAbsolutePath(), swappedFlowFiles, flowFileQueue);
-                                    flowFileQueue.putSwappedRecords(swappedFlowFiles);
-                                }
-
-                                if (!swapFile.delete()) {
-                                    warn("Swapped in FlowFiles from file " + swapFile.getAbsolutePath() + " but failed to delete the file; this file should be cleaned up manually");
-                                }
-                            } catch (final EOFException eof) {
-                                error("Failed to Swap In FlowFiles for " + flowFileQueue + " due to: Corrupt Swap File; will remove this Swap File: " + swapFile);
-                                
-                                if ( !swapFile.delete() ) {
-                                    warn("Failed to remove corrupt Swap File " + swapFile + "; This file should be cleaned up manually");
-                                }
-                            } catch (final FileNotFoundException fnfe) {
-                                error("Failed to Swap In FlowFiles for " + flowFileQueue + " due to: Could not find Swap File " + swapFile);
-                            } catch (final Exception e) {
-                                error("Failed to Swap In FlowFiles for " + flowFileQueue + " due to " + e, e);
-                                
-                                if (swapFile != null) {
-                                    queue.add(swapFile);
-                                }
-                            }
-                        }
-                    } finally {
-                        queueLockWrapper.getLock().unlock();
-                    }
-                }
-            }
-        }
-    }
-
-    private void error(final String error, final Throwable t) {
-        error(error);
-        if ( logger.isDebugEnabled() ) {
-            logger.error("", t);
-        }
-    }
-    
-    private void error(final String error) {
-        logger.error(error);
-        if ( eventReporter != null ) {
-            eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, error);
-        }
-    }
-    
-    private void warn(final String warning) {
-        logger.warn(warning);
-        if ( eventReporter != null ) {
-            eventReporter.reportEvent(Severity.WARNING, EVENT_CATEGORY, warning);
-        }
-    }
-    
-    
-    private class SwapOutTask implements Runnable {
-        private final BlockingQueue<FlowFileQueue> connectionQueue;
-
-        public SwapOutTask(final BlockingQueue<FlowFileQueue> connectionQueue) {
-            this.connectionQueue = connectionQueue;
-        }
-
-        @Override
-        public void run() {
-            while (true) {
-                final FlowFileQueue flowFileQueue = connectionQueue.poll();
-                if (flowFileQueue == null) {
-                    logger.debug("No more FlowFile Queues to Swap Out");
-                    return;
-                }
-
-                if (logger.isDebugEnabled()) {
-                    logger.debug("{} has {} FlowFiles to swap out", flowFileQueue, flowFileQueue.getSwapQueueSize());
-                }
-
-                while (flowFileQueue.getSwapQueueSize() >= MINIMUM_SWAP_COUNT) {
-                    final File swapFile = new File(storageDirectory, System.currentTimeMillis() + "-" + UUID.randomUUID().toString() + ".swap");
-                    final File swapTempFile = new File(swapFile.getParentFile(), swapFile.getName() + ".part");
-                    final String swapLocation = swapFile.getAbsolutePath();
-                    final List<FlowFileRecord> toSwap = flowFileQueue.pollSwappableRecords();
-
-                    int recordsSwapped;
-                    try {
-                        try (final FileOutputStream fos = new FileOutputStream(swapTempFile)) {
-                            recordsSwapped = serializeFlowFiles(toSwap, flowFileQueue, swapLocation, fos);
-                            fos.getFD().sync();
-                        }
-                        
-                        if ( swapTempFile.renameTo(swapFile) ) {
-                            flowFileRepository.swapFlowFilesOut(toSwap, flowFileQueue, swapLocation);
-                        } else {
-                            error("Failed to swap out FlowFiles from " + flowFileQueue + " due to: Unable to rename swap file from " + swapTempFile + " to " + swapFile);
-                            recordsSwapped = 0;
-                        }
-                    } catch (final IOException ioe) {
-                        recordsSwapped = 0;
-                        flowFileQueue.putSwappedRecords(toSwap);
-                        error("Failed to swap out " + toSwap.size() + " FlowFiles from " + flowFileQueue + " to Swap File " + swapLocation + " due to " + ioe, ioe);
-                    }
-
-                    if (recordsSwapped > 0) {
-                        QueueLockWrapper swapQueue = swapMap.get(flowFileQueue);
-                        if (swapQueue == null) {
-                            swapQueue = new QueueLockWrapper(new LinkedBlockingQueue<File>());
-                            QueueLockWrapper oldQueue = swapMap.putIfAbsent(flowFileQueue, swapQueue);
-                            if (oldQueue != null) {
-                                swapQueue = oldQueue;
-                            }
-                        }
-
-                        swapQueue.getQueue().add(swapFile);
-                    } else {
-                        swapTempFile.delete();
-                    }
-                }
-            }
-        }
-    }
-
-    /**
-     * Recovers FlowFiles from all Swap Files, returning the largest FlowFile ID
-     * that was recovered.
-     *
-     * @param queueProvider
-     * @return
-     */
-    @Override
-    public long recoverSwappedFlowFiles(final QueueProvider queueProvider, final ContentClaimManager claimManager) {
-        final File[] swapFiles = storageDirectory.listFiles(new FilenameFilter() {
-            @Override
-            public boolean accept(final File dir, final String name) {
-                return SWAP_FILE_PATTERN.matcher(name).matches() || TEMP_SWAP_FILE_PATTERN.matcher(name).matches();
-            }
-        });
-
-        if (swapFiles == null) {
-            return 0L;
-        }
-
-        final Collection<FlowFileQueue> allQueues = queueProvider.getAllQueues();
-        final Map<String, FlowFileQueue> queueMap = new HashMap<>();
-        for (final FlowFileQueue queue : allQueues) {
-            queueMap.put(queue.getIdentifier(), queue);
-        }
-
-        final ConnectionSwapInfo swapInfo = new ConnectionSwapInfo();
-        int swappedCount = 0;
-        long swappedBytes = 0L;
-        long maxRecoveredId = 0L;
-
-        for (final File swapFile : swapFiles) {
-            if ( TEMP_SWAP_FILE_PATTERN.matcher(swapFile.getName()).matches() ) {
-                if ( swapFile.delete() ) {
-                    logger.info("Removed incomplete/temporary Swap File " + swapFile);
-                } else {
-                    warn("Failed to remove incomplete/temporary Swap File " + swapFile + "; this file should be cleaned up manually");
-                }
-                
-                continue;
-            }
-            
-            // read record to disk via the swap file
-            try (final InputStream fis = new FileInputStream(swapFile);
-                    final InputStream bufferedIn = new BufferedInputStream(fis);
-                    final DataInputStream in = new DataInputStream(bufferedIn)) {
-
-                final int swapEncodingVersion = in.readInt();
-                if (swapEncodingVersion > SWAP_ENCODING_VERSION) {
-                    final String errMsg = "Cannot swap FlowFiles in from " + swapFile + " because the encoding version is "
-                            + swapEncodingVersion + ", which is too new (expecting " + SWAP_ENCODING_VERSION + " or less)";
-
-                    eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, errMsg);
-                    throw new IOException(errMsg);
-                }
-
-                final String connectionId = in.readUTF();
-                final FlowFileQueue queue = queueMap.get(connectionId);
-                if (queue == null) {
-                    error("Cannot recover Swapped FlowFiles from Swap File " + swapFile + " because the FlowFiles belong to a Connection with ID " + connectionId + " and that Connection does not exist");
-                    continue;
-                }
-
-                final int numRecords = in.readInt();
-                final long contentSize = in.readLong();
-
-                swapInfo.addSwapSizeInfo(connectionId, swapFile.getAbsolutePath(), new QueueSize(numRecords, contentSize));
-                swappedCount += numRecords;
-                swappedBytes += contentSize;
-
-                final List<FlowFileRecord> records = deserializeFlowFiles(in, numRecords, queue, swapEncodingVersion, true, claimManager);
-                long maxId = 0L;
-                for (final FlowFileRecord record : records) {
-                    if (record.getId() > maxId) {
-                        maxId = record.getId();
-                    }
-                }
-
-                if (maxId > maxRecoveredId) {
-                    maxRecoveredId = maxId;
-                }
-            } catch (final IOException ioe) {
-                error("Cannot recover Swapped FlowFiles from Swap File " + swapFile + " due to " + ioe, ioe);
-            }
-        }
-
-        restoreSwapLocations(queueMap.values(), swapInfo);
-        logger.info("Recovered {} FlowFiles ({} bytes) from Swap Files", swappedCount, swappedBytes);
-        return maxRecoveredId;
-    }
-
-    public void restoreSwapLocations(final Collection<FlowFileQueue> flowFileQueues, final ConnectionSwapInfo swapInfo) {
-        for (final FlowFileQueue queue : flowFileQueues) {
-            final String queueId = queue.getIdentifier();
-            final Collection<String> swapFileLocations = swapInfo.getSwapFileLocations(queueId);
-            if (swapFileLocations == null || swapFileLocations.isEmpty()) {
-                continue;
-            }
-
-            final SortedMap<String, QueueSize> sortedFileQueueMap = new TreeMap<>(new SwapFileComparator());
-            for (final String swapFileLocation : swapFileLocations) {
-                final QueueSize queueSize = swapInfo.getSwappedSize(queueId, swapFileLocation);
-                sortedFileQueueMap.put(swapFileLocation, queueSize);
-            }
-
-            QueueLockWrapper fileQueue = swapMap.get(queue);
-            if (fileQueue == null) {
-                fileQueue = new QueueLockWrapper(new LinkedBlockingQueue<File>());
-                swapMap.put(queue, fileQueue);
-            }
-
-            for (final Map.Entry<String, QueueSize> innerEntry : sortedFileQueueMap.entrySet()) {
-                final File swapFile = new File(innerEntry.getKey());
-                final QueueSize size = innerEntry.getValue();
-                fileQueue.getQueue().add(swapFile);
-                queue.incrementSwapCount(size.getObjectCount(), size.getByteCount());
-            }
-        }
-    }
-
-    public void shutdown() {
-        swapQueueIdentifierExecutor.shutdownNow();
-        swapInExecutor.shutdownNow();
-    }
-
-    private static class SwapFileComparator implements Comparator<String> {
-
-        @Override
-        public int compare(final String o1, final String o2) {
-            if (o1 == o2) {
-                return 0;
-            }
-
-            final Long time1 = getTimestampFromFilename(o1);
-            final Long time2 = getTimestampFromFilename(o2);
-
-            if (time1 == null && time2 == null) {
-                return 0;
-            }
-            if (time1 == null) {
-                return 1;
-            }
-            if (time2 == null) {
-                return -1;
-            }
-
-            final int timeComparisonValue = time1.compareTo(time2);
-            if (timeComparisonValue != 0) {
-                return timeComparisonValue;
-            }
-
-            return o1.compareTo(o2);
-        }
-
-        private Long getTimestampFromFilename(final String fullyQualifiedFilename) {
-            if (fullyQualifiedFilename == null) {
-                return null;
-            }
-
-            final File file = new File(fullyQualifiedFilename);
-            final String filename = file.getName();
-
-            final int idx = filename.indexOf("-");
-            if (idx < 1) {
-                return null;
-            }
-
-            final String millisVal = filename.substring(0, idx);
-            try {
-                return Long.parseLong(millisVal);
-            } catch (final NumberFormatException e) {
-                return null;
-            }
-        }
-    }
-
-    private static class QueueLockWrapper {
-
-        private final Lock lock = new ReentrantLock();
-        private final Queue<File> queue;
-
-        public QueueLockWrapper(final Queue<File> queue) {
-            this.queue = queue;
-        }
-
-        public Queue<File> getQueue() {
-            return queue;
-        }
-
-        public Lock getLock() {
-            return lock;
-        }
-
-        @Override
-        public int hashCode() {
-            return queue.hashCode();
-        }
-
-        @Override
-        public boolean equals(final Object obj) {
-            if (obj instanceof QueueLockWrapper) {
-                return queue.equals(((QueueLockWrapper) obj).queue);
-            }
-            return false;
-        }
-    }
-}


[04/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowFromDOMFactory.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowFromDOMFactory.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowFromDOMFactory.java
deleted file mode 100644
index c67181a..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowFromDOMFactory.java
+++ /dev/null
@@ -1,418 +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;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.connectable.Size;
-import org.apache.nifi.encrypt.StringEncryptor;
-import org.apache.nifi.groups.RemoteProcessGroupPortDescriptor;
-import org.apache.nifi.remote.StandardRemoteProcessGroupPortDescriptor;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.util.DomUtils;
-import org.apache.nifi.web.api.dto.ConnectableDTO;
-import org.apache.nifi.web.api.dto.ConnectionDTO;
-import org.apache.nifi.web.api.dto.FlowSnippetDTO;
-import org.apache.nifi.web.api.dto.FunnelDTO;
-import org.apache.nifi.web.api.dto.LabelDTO;
-import org.apache.nifi.web.api.dto.PortDTO;
-import org.apache.nifi.web.api.dto.PositionDTO;
-import org.apache.nifi.web.api.dto.ProcessGroupDTO;
-import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
-import org.apache.nifi.web.api.dto.ProcessorDTO;
-import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
-
-import org.w3c.dom.Element;
-import org.w3c.dom.NodeList;
-
-public class FlowFromDOMFactory {
-
-    public static PositionDTO getPosition(final Element positionElement) {
-        if (positionElement == null) {
-            throw new IllegalArgumentException("Invalid Flow: Found no 'position' element");
-        }
-        return new PositionDTO(Double.parseDouble(positionElement.getAttribute("x")), Double.parseDouble(positionElement.getAttribute("y")));
-    }
-
-    public static Size getSize(final Element sizeElement) {
-        if (sizeElement == null) {
-            throw new IllegalArgumentException("Invalid Flow: Found no 'size' element");
-        }
-
-        return new Size(Double.parseDouble(sizeElement.getAttribute("width")), Double.parseDouble(sizeElement.getAttribute("height")));
-    }
-
-    public static Map<String, String> getStyle(final Element stylesElement) {
-        final Map<String, String> styles = new HashMap<>();
-        if (stylesElement == null) {
-            return styles;
-        }
-
-        for (final Element styleElement : getChildrenByTagName(stylesElement, "style")) {
-            final String styleName = styleElement.getAttribute("name");
-            final String styleValue = styleElement.getTextContent();
-            styles.put(styleName, styleValue);
-        }
-
-        return styles;
-    }
-
-    public static ProcessGroupDTO getProcessGroup(final String parentId, final Element element, final StringEncryptor encryptor) {
-        final ProcessGroupDTO dto = new ProcessGroupDTO();
-        final String groupId = getString(element, "id");
-        dto.setId(groupId);
-        dto.setParentGroupId(parentId);
-        dto.setName(getString(element, "name"));
-        dto.setPosition(getPosition(DomUtils.getChild(element, "position")));
-        dto.setComments(getString(element, "comment"));
-
-        final Set<ProcessorDTO> processors = new HashSet<>();
-        final Set<ConnectionDTO> connections = new HashSet<>();
-        final Set<FunnelDTO> funnels = new HashSet<>();
-        final Set<PortDTO> inputPorts = new HashSet<>();
-        final Set<PortDTO> outputPorts = new HashSet<>();
-        final Set<LabelDTO> labels = new HashSet<>();
-        final Set<ProcessGroupDTO> processGroups = new HashSet<>();
-        final Set<RemoteProcessGroupDTO> remoteProcessGroups = new HashSet<>();
-
-        final FlowSnippetDTO groupContents = new FlowSnippetDTO();
-        groupContents.setConnections(connections);
-        groupContents.setFunnels(funnels);
-        groupContents.setInputPorts(inputPorts);
-        groupContents.setLabels(labels);
-        groupContents.setOutputPorts(outputPorts);
-        groupContents.setProcessGroups(processGroups);
-        groupContents.setProcessors(processors);
-        groupContents.setRemoteProcessGroups(remoteProcessGroups);
-
-        NodeList nodeList = DomUtils.getChildNodesByTagName(element, "processor");
-        for (int i = 0; i < nodeList.getLength(); i++) {
-            processors.add(getProcessor((Element) nodeList.item(i), encryptor));
-        }
-
-        nodeList = DomUtils.getChildNodesByTagName(element, "funnel");
-        for (int i = 0; i < nodeList.getLength(); i++) {
-            funnels.add(getFunnel((Element) nodeList.item(i)));
-        }
-
-        nodeList = DomUtils.getChildNodesByTagName(element, "inputPort");
-        for (int i = 0; i < nodeList.getLength(); i++) {
-            inputPorts.add(getPort((Element) nodeList.item(i)));
-        }
-
-        nodeList = DomUtils.getChildNodesByTagName(element, "outputPort");
-        for (int i = 0; i < nodeList.getLength(); i++) {
-            outputPorts.add(getPort((Element) nodeList.item(i)));
-        }
-
-        nodeList = DomUtils.getChildNodesByTagName(element, "label");
-        for (int i = 0; i < nodeList.getLength(); i++) {
-            labels.add(getLabel((Element) nodeList.item(i)));
-        }
-
-        nodeList = DomUtils.getChildNodesByTagName(element, "processGroup");
-        for (int i = 0; i < nodeList.getLength(); i++) {
-            processGroups.add(getProcessGroup(groupId, (Element) nodeList.item(i), encryptor));
-        }
-
-        nodeList = DomUtils.getChildNodesByTagName(element, "remoteProcessGroup");
-        for (int i = 0; i < nodeList.getLength(); i++) {
-            remoteProcessGroups.add(getRemoteProcessGroup((Element) nodeList.item(i)));
-        }
-
-        nodeList = DomUtils.getChildNodesByTagName(element, "connection");
-        for (int i = 0; i < nodeList.getLength(); i++) {
-            connections.add(getConnection((Element) nodeList.item(i)));
-        }
-
-        dto.setContents(groupContents);
-        return dto;
-    }
-
-    public static ConnectionDTO getConnection(final Element element) {
-        final ConnectionDTO dto = new ConnectionDTO();
-        dto.setId(getString(element, "id"));
-        dto.setName(getString(element, "name"));
-        dto.setLabelIndex(getOptionalInt(element, "labelIndex"));
-        dto.setzIndex(getOptionalLong(element, "zIndex"));
-
-        final List<PositionDTO> bends = new ArrayList<>();
-        final Element bendPointsElement = DomUtils.getChild(element, "bendPoints");
-        if (bendPointsElement != null) {
-            for (final Element bendPointElement : getChildrenByTagName(bendPointsElement, "bendPoint")) {
-                final PositionDTO bend = getPosition(bendPointElement);
-                bends.add(bend);
-            }
-        }
-        dto.setBends(bends);
-
-        final ConnectableDTO sourceConnectable = new ConnectableDTO();
-        dto.setSource(sourceConnectable);
-        sourceConnectable.setId(getString(element, "sourceId"));
-        sourceConnectable.setGroupId(getString(element, "sourceGroupId"));
-        sourceConnectable.setType(getString(element, "sourceType"));
-
-        final ConnectableDTO destConnectable = new ConnectableDTO();
-        dto.setDestination(destConnectable);
-        destConnectable.setId(getString(element, "destinationId"));
-        destConnectable.setGroupId(getString(element, "destinationGroupId"));
-        destConnectable.setType(getString(element, "destinationType"));
-
-        final Set<String> relationships = new HashSet<>();
-        final List<Element> relationshipNodeList = getChildrenByTagName(element, "relationship");
-        for (final Element relationshipElem : relationshipNodeList) {
-            relationships.add(relationshipElem.getTextContent());
-        }
-        dto.setSelectedRelationships(relationships);
-
-        dto.setBackPressureObjectThreshold(getLong(element, "maxWorkQueueSize"));
-
-        final String maxDataSize = getString(element, "maxWorkQueueDataSize");
-        if (maxDataSize != null && !maxDataSize.trim().isEmpty()) {
-            dto.setBackPressureDataSizeThreshold(maxDataSize);
-        }
-
-        String expiration = getString(element, "flowFileExpiration");
-        if (expiration == null) {
-            expiration = "0 sec";
-        }
-        dto.setFlowFileExpiration(expiration);
-
-        final List<String> prioritizerClasses = new ArrayList<>();
-        final List<Element> prioritizerNodeList = getChildrenByTagName(element, "queuePrioritizerClass");
-        for (final Element prioritizerElement : prioritizerNodeList) {
-            prioritizerClasses.add(prioritizerElement.getTextContent().trim());
-        }
-        dto.setPrioritizers(prioritizerClasses);
-
-        return dto;
-    }
-
-    public static RemoteProcessGroupDTO getRemoteProcessGroup(final Element element) {
-        final RemoteProcessGroupDTO dto = new RemoteProcessGroupDTO();
-        dto.setId(getString(element, "id"));
-        dto.setName(getString(element, "name"));
-        dto.setTargetUri(getString(element, "url"));
-        dto.setTransmitting(getBoolean(element, "transmitting"));
-        dto.setPosition(getPosition(DomUtils.getChild(element, "position")));
-        dto.setCommunicationsTimeout(getString(element, "timeout"));
-        dto.setComments(getString(element, "comment"));
-
-        return dto;
-    }
-
-    public static LabelDTO getLabel(final Element element) {
-        final LabelDTO dto = new LabelDTO();
-        dto.setId(getString(element, "id"));
-        dto.setLabel(getString(element, "value"));
-        dto.setPosition(getPosition(DomUtils.getChild(element, "position")));
-        final Size size = getSize(DomUtils.getChild(element, "size"));
-        dto.setWidth(size.getWidth());
-        dto.setHeight(size.getHeight());
-        dto.setStyle(getStyle(DomUtils.getChild(element, "styles")));
-
-        return dto;
-    }
-
-    public static FunnelDTO getFunnel(final Element element) {
-        final FunnelDTO dto = new FunnelDTO();
-        dto.setId(getString(element, "id"));
-        dto.setPosition(getPosition(DomUtils.getChild(element, "position")));
-
-        return dto;
-    }
-
-    public static PortDTO getPort(final Element element) {
-        final PortDTO portDTO = new PortDTO();
-        portDTO.setId(getString(element, "id"));
-        portDTO.setPosition(getPosition(DomUtils.getChild(element, "position")));
-        portDTO.setName(getString(element, "name"));
-        portDTO.setComments(getString(element, "comments"));
-        final ScheduledState scheduledState = getScheduledState(element);
-        portDTO.setState(scheduledState.toString());
-
-        final List<Element> maxTasksElements = getChildrenByTagName(element, "maxConcurrentTasks");
-        if (!maxTasksElements.isEmpty()) {
-            portDTO.setConcurrentlySchedulableTaskCount(Integer.parseInt(maxTasksElements.get(0).getTextContent()));
-        }
-
-        final List<Element> userAccessControls = getChildrenByTagName(element, "userAccessControl");
-        if (userAccessControls != null && !userAccessControls.isEmpty()) {
-            final Set<String> users = new HashSet<>();
-            portDTO.setUserAccessControl(users);
-            for (final Element userElement : userAccessControls) {
-                users.add(userElement.getTextContent());
-            }
-        }
-
-        final List<Element> groupAccessControls = getChildrenByTagName(element, "groupAccessControl");
-        if (groupAccessControls != null && !groupAccessControls.isEmpty()) {
-            final Set<String> groups = new HashSet<>();
-            portDTO.setGroupAccessControl(groups);
-            for (final Element groupElement : groupAccessControls) {
-                groups.add(groupElement.getTextContent());
-            }
-        }
-
-        return portDTO;
-    }
-
-    public static RemoteProcessGroupPortDescriptor getRemoteProcessGroupPort(final Element element) {
-        final StandardRemoteProcessGroupPortDescriptor descriptor = new StandardRemoteProcessGroupPortDescriptor();
-
-        // What we have serialized is the ID of the Remote Process Group, followed by a dash ('-'), followed by
-        // the actual ID of the port; we want to get rid of the remote process group id.
-        String id = getString(element, "id");
-        if (id.length() > 37) {
-            id = id.substring(37);
-        }
-
-        descriptor.setId(id);
-        descriptor.setName(getString(element, "name"));
-        descriptor.setComments(getString(element, "comments"));
-        descriptor.setConcurrentlySchedulableTaskCount(getInt(element, "maxConcurrentTasks"));
-        descriptor.setUseCompression(getBoolean(element, "useCompression"));
-        descriptor.setTransmitting("RUNNING".equalsIgnoreCase(getString(element, "scheduledState")));
-
-        return descriptor;
-    }
-
-    public static ProcessorDTO getProcessor(final Element element, final StringEncryptor encryptor) {
-        final ProcessorDTO dto = new ProcessorDTO();
-
-        dto.setId(getString(element, "id"));
-        dto.setName(getString(element, "name"));
-        dto.setType(getString(element, "class"));
-        dto.setPosition(getPosition(DomUtils.getChild(element, "position")));
-        dto.setStyle(getStyle(DomUtils.getChild(element, "styles")));
-
-        final ProcessorConfigDTO configDto = new ProcessorConfigDTO();
-        dto.setConfig(configDto);
-        configDto.setComments(getString(element, "comment"));
-        configDto.setAnnotationData(getString(element, "annotationData"));
-        configDto.setConcurrentlySchedulableTaskCount(getInt(element, "maxConcurrentTasks"));
-        final String schedulingPeriod = getString(element, "schedulingPeriod");
-        configDto.setSchedulingPeriod(schedulingPeriod);
-        configDto.setPenaltyDuration(getString(element, "penalizationPeriod"));
-        configDto.setYieldDuration(getString(element, "yieldPeriod"));
-        configDto.setBulletinLevel(getString(element, "bulletinLevel"));
-        configDto.setLossTolerant(getBoolean(element, "lossTolerant"));
-        final ScheduledState scheduledState = getScheduledState(element);
-        dto.setState(scheduledState.toString());
-
-        // handle scheduling strategy
-        final String schedulingStrategyName = getString(element, "schedulingStrategy");
-        if (schedulingStrategyName == null || schedulingStrategyName.trim().isEmpty()) {
-            configDto.setSchedulingStrategy(SchedulingStrategy.TIMER_DRIVEN.name());
-        } else {
-            configDto.setSchedulingStrategy(schedulingStrategyName.trim());
-        }
-
-        final Long runDurationNanos = getOptionalLong(element, "runDurationNanos");
-        if (runDurationNanos != null) {
-            configDto.setRunDurationMillis(TimeUnit.NANOSECONDS.toMillis(runDurationNanos));
-        }
-
-        final LinkedHashMap<String, String> properties = new LinkedHashMap<>();
-        final List<Element> propertyNodeList = getChildrenByTagName(element, "property");
-        for (final Element propertyElement : propertyNodeList) {
-            final String name = getString(propertyElement, "name");
-            final String value = decrypt(getString(propertyElement, "value"), encryptor);
-            properties.put(name, value);
-        }
-        configDto.setProperties(properties);
-
-        final Set<String> autoTerminatedRelationships = new HashSet<>();
-        final List<Element> autoTerminateList = getChildrenByTagName(element, "autoTerminatedRelationship");
-        for (final Element autoTerminateElement : autoTerminateList) {
-            autoTerminatedRelationships.add(autoTerminateElement.getTextContent());
-        }
-        configDto.setAutoTerminatedRelationships(autoTerminatedRelationships);
-
-        return dto;
-    }
-
-    private static String getString(final Element element, final String childElementName) {
-        final List<Element> nodeList = getChildrenByTagName(element, childElementName);
-        if (nodeList == null || nodeList.isEmpty()) {
-            return null;
-        }
-        final Element childElement = nodeList.get(0);
-        return childElement.getTextContent();
-    }
-
-    private static Integer getOptionalInt(final Element element, final String childElementName) {
-        final List<Element> nodeList = getChildrenByTagName(element, childElementName);
-        if (nodeList == null || nodeList.isEmpty()) {
-            return null;
-        }
-        final Element childElement = nodeList.get(0);
-        final String val = childElement.getTextContent();
-        if (val == null) {
-            return null;
-        }
-        return Integer.parseInt(val);
-    }
-
-    private static Long getOptionalLong(final Element element, final String childElementName) {
-        final List<Element> nodeList = getChildrenByTagName(element, childElementName);
-        if (nodeList == null || nodeList.isEmpty()) {
-            return null;
-        }
-        final Element childElement = nodeList.get(0);
-        final String val = childElement.getTextContent();
-        if (val == null) {
-            return null;
-        }
-        return Long.parseLong(val);
-    }
-
-    private static int getInt(final Element element, final String childElementName) {
-        return Integer.parseInt(getString(element, childElementName));
-    }
-
-    private static long getLong(final Element element, final String childElementName) {
-        return Long.parseLong(getString(element, childElementName));
-    }
-
-    private static boolean getBoolean(final Element element, final String childElementName) {
-        return Boolean.parseBoolean(getString(element, childElementName));
-    }
-
-    private static ScheduledState getScheduledState(final Element element) {
-        return ScheduledState.valueOf(getString(element, "scheduledState"));
-    }
-
-    private static List<Element> getChildrenByTagName(final Element element, final String childElementName) {
-        return DomUtils.getChildElementsByTagName(element, childElementName);
-    }
-
-    private static String decrypt(final String value, final StringEncryptor encryptor) {
-        if (value != null && value.startsWith(FlowSerializer.ENC_PREFIX) && value.endsWith(FlowSerializer.ENC_SUFFIX)) {
-            return encryptor.decrypt(value.substring(FlowSerializer.ENC_PREFIX.length(), value.length() - FlowSerializer.ENC_SUFFIX.length()));
-        } else {
-            return value;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSerializationException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSerializationException.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSerializationException.java
deleted file mode 100644
index f1ee760..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSerializationException.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.controller;
-
-/**
- * Represents the exceptional case when flow configuration is malformed and
- * therefore, cannot be serialized or deserialized.
- *
- * @author unattributed
- */
-public class FlowSerializationException extends RuntimeException {
-
-    private static final long serialVersionUID = 128934798237L;
-
-    public FlowSerializationException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
-        super(message, cause, enableSuppression, writableStackTrace);
-    }
-
-    public FlowSerializationException(Throwable cause) {
-        super(cause);
-    }
-
-    public FlowSerializationException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public FlowSerializationException(String message) {
-        super(message);
-    }
-
-    public FlowSerializationException() {
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSerializer.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSerializer.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSerializer.java
deleted file mode 100644
index 331b26c..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSerializer.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.controller;
-
-import java.io.OutputStream;
-
-/**
- * Serializes the flow configuration of a controller instance to an output
- * stream.
- *
- * @author unattributed
- */
-public interface FlowSerializer {
-
-    public static final String ENC_PREFIX = "enc{";
-    public static final String ENC_SUFFIX = "}";
-
-    /**
-     * Serializes the flow configuration of a controller instance.
-     *
-     * @param controller a controller
-     * @param os an output stream to write the configuration to
-     *
-     * @throws FlowSerializationException if serialization failed
-     */
-    void serialize(FlowController controller, OutputStream os) throws FlowSerializationException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSynchronizationException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSynchronizationException.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSynchronizationException.java
deleted file mode 100644
index 706ac46..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSynchronizationException.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.controller;
-
-/**
- * Represents the exceptional case when a controller managing an existing flow
- * fails to fully load a different flow.
- *
- * @author unattributed
- */
-public class FlowSynchronizationException extends RuntimeException {
-
-    private static final long serialVersionUID = 109234802938L;
-
-    public FlowSynchronizationException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
-        super(message, cause, enableSuppression, writableStackTrace);
-    }
-
-    public FlowSynchronizationException(Throwable cause) {
-        super(cause);
-    }
-
-    public FlowSynchronizationException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public FlowSynchronizationException(String message) {
-        super(message);
-    }
-
-    public FlowSynchronizationException() {
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSynchronizer.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSynchronizer.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSynchronizer.java
deleted file mode 100644
index f6889fe..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSynchronizer.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.controller;
-
-import org.apache.nifi.cluster.protocol.DataFlow;
-import org.apache.nifi.encrypt.StringEncryptor;
-
-/**
- * @author unattributed
- */
-public interface FlowSynchronizer {
-
-    /**
-     * Synchronizes the given controller with the given flow configuration. If
-     * loading the proposed flow configuration would cause the controller to
-     * orphan flow files, then an UninheritableFlowException is thrown.
-     *
-     * If the FlowSynchronizationException is thrown, then the controller may
-     * have changed some of its state and should no longer be used.
-     *
-     * @param controller the flow controller
-     * @param dataFlow the flow to load the controller with. If the flow is null
-     * or zero length, then the controller must not have a flow or else an
-     * UninheritableFlowException will be thrown.
-     * @param encryptor used for the encryption/decryption of sensitive property
-     * values
-     *
-     * @throws FlowSerializationException if proposed flow is not a valid flow
-     * configuration file
-     * @throws UninheritableFlowException if the proposed flow cannot be loaded
-     * by the controller because in doing so would risk orphaning flow files
-     * @throws FlowSynchronizationException if updates to the controller failed.
-     * If this exception is thrown, then the controller should be considered
-     * unsafe to be used
-     */
-    void sync(FlowController controller, DataFlow dataFlow, StringEncryptor encryptor)
-            throws FlowSerializationException, UninheritableFlowException, FlowSynchronizationException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowUnmarshaller.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowUnmarshaller.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowUnmarshaller.java
deleted file mode 100644
index 42d7f1c..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowUnmarshaller.java
+++ /dev/null
@@ -1,78 +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;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Objects;
-import java.util.Set;
-
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-
-import org.apache.nifi.encrypt.StringEncryptor;
-import org.apache.nifi.stream.io.ByteArrayInputStream;
-import org.apache.nifi.web.api.dto.FlowSnippetDTO;
-import org.apache.nifi.web.api.dto.ProcessGroupDTO;
-
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.NodeList;
-import org.xml.sax.SAXException;
-
-public class FlowUnmarshaller {
-
-    /**
-     * Interprets the given byte array as an XML document that conforms to the
-     * Flow Configuration schema and returns a FlowSnippetDTO representing the
-     * flow
-     *
-     * @param flowContents
-     * @param encryptor
-     * @return
-     * @throws NullPointerException if <code>flowContents</code> is null
-     * @throws IOException
-     * @throws SAXException
-     * @throws ParserConfigurationException
-     */
-    public static FlowSnippetDTO unmarshal(final byte[] flowContents, final StringEncryptor encryptor) throws IOException, SAXException, ParserConfigurationException {
-        if (Objects.requireNonNull(flowContents).length == 0) {
-            return new FlowSnippetDTO();
-        }
-
-        final DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
-        final DocumentBuilder docBuilder = dbf.newDocumentBuilder();
-
-        final Document document = docBuilder.parse(new ByteArrayInputStream(flowContents));
-        final FlowSnippetDTO flowDto = new FlowSnippetDTO();
-
-        final NodeList nodeList = document.getElementsByTagName("rootGroup");
-        if (nodeList.getLength() == 0) {
-            return flowDto;
-        }
-        if (nodeList.getLength() > 1) {
-            throw new IllegalArgumentException("Contents contain multiple rootGroup elements");
-        }
-
-        final Set<ProcessGroupDTO> rootGroupSet = new HashSet<>();
-        flowDto.setProcessGroups(rootGroupSet);
-        rootGroupSet.add(FlowFromDOMFactory.getProcessGroup(null, (Element) nodeList.item(0), encryptor));
-
-        return flowDto;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/SnippetManager.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/SnippetManager.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/SnippetManager.java
deleted file mode 100644
index 3a9662e..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/SnippetManager.java
+++ /dev/null
@@ -1,96 +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;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.nifi.stream.io.ByteArrayInputStream;
-import org.apache.nifi.stream.io.ByteArrayOutputStream;
-import org.apache.nifi.stream.io.DataOutputStream;
-import org.apache.nifi.stream.io.StreamUtils;
-import org.apache.nifi.persistence.StandardSnippetDeserializer;
-import org.apache.nifi.persistence.StandardSnippetSerializer;
-
-public class SnippetManager {
-
-    private final ConcurrentMap<String, StandardSnippet> snippetMap = new ConcurrentHashMap<>();
-
-    public void addSnippet(final StandardSnippet snippet) {
-        final StandardSnippet oldSnippet = this.snippetMap.putIfAbsent(snippet.getId(), snippet);
-        if (oldSnippet != null) {
-            throw new IllegalStateException("Snippet with ID " + snippet.getId() + " already exists");
-        }
-    }
-
-    public void removeSnippet(final StandardSnippet snippet) {
-        if (!snippetMap.remove(snippet.getId(), snippet)) {
-            throw new IllegalStateException("Snippet is not contained in this SnippetManager");
-        }
-    }
-
-    public StandardSnippet getSnippet(final String identifier) {
-        return snippetMap.get(identifier);
-    }
-
-    public Collection<StandardSnippet> getSnippets() {
-        return snippetMap.values();
-    }
-
-    public void clear() {
-        snippetMap.clear();
-    }
-
-    public static List<StandardSnippet> parseBytes(final byte[] bytes) {
-        final List<StandardSnippet> snippets = new ArrayList<>();
-
-        try (final InputStream rawIn = new ByteArrayInputStream(bytes);
-                final DataInputStream in = new DataInputStream(rawIn)) {
-            final int length = in.readInt();
-            final byte[] buffer = new byte[length];
-            StreamUtils.fillBuffer(in, buffer, true);
-            final StandardSnippet snippet = StandardSnippetDeserializer.deserialize(new ByteArrayInputStream(buffer));
-            snippets.add(snippet);
-        } catch (final IOException e) {
-            throw new RuntimeException("Failed to parse bytes", e);  // should never happen because of streams being used
-        }
-
-        return snippets;
-    }
-
-    public byte[] export() {
-        try (final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-                final DataOutputStream dos = new DataOutputStream(baos)) {
-            for (final StandardSnippet snippet : getSnippets()) {
-                final byte[] bytes = StandardSnippetSerializer.serialize(snippet);
-                dos.writeInt(bytes.length);
-                dos.write(bytes);
-            }
-
-            return baos.toByteArray();
-        } catch (final IOException e) {
-            // won't happen
-            return null;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardCounter.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardCounter.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardCounter.java
deleted file mode 100644
index 2899a85..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardCounter.java
+++ /dev/null
@@ -1,108 +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;
-
-import org.apache.nifi.controller.Counter;
-import java.util.concurrent.atomic.AtomicLong;
-
-public class StandardCounter implements Counter {
-
-    private final String identifier;
-    private final String context;
-    private final String name;
-    private final AtomicLong value;
-
-    public StandardCounter(final String identifier, final String context, final String name) {
-        this.identifier = identifier;
-        this.context = context;
-        this.name = name;
-        this.value = new AtomicLong(0L);
-    }
-
-    public void adjust(final long delta) {
-        this.value.addAndGet(delta);
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public long getValue() {
-        return this.value.get();
-    }
-
-    public String getContext() {
-        return context;
-    }
-
-    public String getIdentifier() {
-        return identifier;
-    }
-
-    public void reset() {
-        this.value.set(0);
-    }
-
-    @Override
-    public String toString() {
-        return "Counter[identifier=" + identifier + ", context=" + context + ", name=" + name + ", value=" + value + ']';
-    }
-
-    public static UnmodifiableCounter unmodifiableCounter(final Counter counter) {
-        return new UnmodifiableCounter(counter);
-    }
-
-    static class UnmodifiableCounter extends StandardCounter {
-
-        private final Counter counter;
-
-        public UnmodifiableCounter(final Counter counter) {
-            super(counter.getIdentifier(), counter.getContext(), counter.getName());
-            this.counter = counter;
-        }
-
-        @Override
-        public void adjust(long delta) {
-            throw new UnsupportedOperationException("Cannot modify value of UnmodifiableCounter");
-        }
-
-        @Override
-        public String getName() {
-            return counter.getName();
-        }
-
-        @Override
-        public long getValue() {
-            return counter.getValue();
-        }
-
-        @Override
-        public String getContext() {
-            return counter.getContext();
-        }
-
-        @Override
-        public String getIdentifier() {
-            return counter.getIdentifier();
-        }
-
-        @Override
-        public String toString() {
-            return counter.toString();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowSerializer.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowSerializer.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowSerializer.java
deleted file mode 100644
index e08a94d..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowSerializer.java
+++ /dev/null
@@ -1,404 +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;
-
-import java.io.BufferedOutputStream;
-import java.io.OutputStream;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.transform.OutputKeys;
-import javax.xml.transform.Transformer;
-import javax.xml.transform.TransformerException;
-import javax.xml.transform.TransformerFactory;
-import javax.xml.transform.TransformerFactoryConfigurationError;
-import javax.xml.transform.dom.DOMSource;
-import javax.xml.transform.stream.StreamResult;
-
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.connectable.ConnectableType;
-import org.apache.nifi.connectable.Connection;
-import org.apache.nifi.connectable.Funnel;
-import org.apache.nifi.connectable.Port;
-import org.apache.nifi.connectable.Position;
-import org.apache.nifi.connectable.Size;
-import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.encrypt.StringEncryptor;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
-import org.apache.nifi.groups.ProcessGroup;
-import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.remote.RemoteGroupPort;
-import org.apache.nifi.remote.RootGroupPort;
-
-import org.w3c.dom.DOMException;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-
-/**
- * Serializes a Flow Controller as XML to an output stream.
- *
- * NOT THREAD-SAFE.
- */
-public class StandardFlowSerializer implements FlowSerializer {
-
-    private final StringEncryptor encryptor;
-
-    public StandardFlowSerializer(final StringEncryptor encryptor) {
-        this.encryptor = encryptor;
-    }
-
-    @Override
-    public void serialize(final FlowController controller, final OutputStream os) throws FlowSerializationException {
-        try {
-            // create a new, empty document
-            final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance();
-            final DocumentBuilder docBuilder = docFactory.newDocumentBuilder();
-            final Document doc = docBuilder.newDocument();
-
-            // populate document with controller state
-            final Element rootNode = doc.createElement("flowController");
-            doc.appendChild(rootNode);
-            addTextElement(rootNode, "maxTimerDrivenThreadCount", controller.getMaxTimerDrivenThreadCount());
-            addTextElement(rootNode, "maxEventDrivenThreadCount", controller.getMaxEventDrivenThreadCount());
-            addProcessGroup(rootNode, controller.getGroup(controller.getRootGroupId()), "rootGroup");
-
-            final DOMSource domSource = new DOMSource(doc);
-            final StreamResult streamResult = new StreamResult(new BufferedOutputStream(os));
-
-            // configure the transformer and convert the DOM
-            final TransformerFactory transformFactory = TransformerFactory.newInstance();
-            final Transformer transformer = transformFactory.newTransformer();
-            transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "2");
-            transformer.setOutputProperty(OutputKeys.INDENT, "yes");
-
-            // transform the document to byte stream
-            transformer.transform(domSource, streamResult);
-
-        } catch (final ParserConfigurationException | DOMException | TransformerFactoryConfigurationError | IllegalArgumentException | TransformerException e) {
-            throw new FlowSerializationException(e);
-        }
-    }
-
-    private void addSize(final Element parentElement, final Size size) {
-        final Element element = parentElement.getOwnerDocument().createElement("size");
-        element.setAttribute("width", String.valueOf(size.getWidth()));
-        element.setAttribute("height", String.valueOf(size.getHeight()));
-        parentElement.appendChild(element);
-    }
-
-    private void addPosition(final Element parentElement, final Position position) {
-        addPosition(parentElement, position, "position");
-    }
-
-    private void addPosition(final Element parentElement, final Position position, final String elementName) {
-        final Element element = parentElement.getOwnerDocument().createElement(elementName);
-        element.setAttribute("x", String.valueOf(position.getX()));
-        element.setAttribute("y", String.valueOf(position.getY()));
-        parentElement.appendChild(element);
-    }
-
-    private void addProcessGroup(final Element parentElement, final ProcessGroup group, final String elementName) {
-        final Document doc = parentElement.getOwnerDocument();
-        final Element element = doc.createElement(elementName);
-        parentElement.appendChild(element);
-        addTextElement(element, "id", group.getIdentifier());
-        addTextElement(element, "name", group.getName());
-        addPosition(element, group.getPosition());
-        addTextElement(element, "comment", group.getComments());
-
-        for (final ProcessorNode processor : group.getProcessors()) {
-            addProcessor(element, processor);
-        }
-
-        if (group.isRootGroup()) {
-            for (final Port port : group.getInputPorts()) {
-                addRootGroupPort(element, (RootGroupPort) port, "inputPort");
-            }
-
-            for (final Port port : group.getOutputPorts()) {
-                addRootGroupPort(element, (RootGroupPort) port, "outputPort");
-            }
-        } else {
-            for (final Port port : group.getInputPorts()) {
-                addPort(element, port, "inputPort");
-            }
-
-            for (final Port port : group.getOutputPorts()) {
-                addPort(element, port, "outputPort");
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            addLabel(element, label);
-        }
-
-        for (final Funnel funnel : group.getFunnels()) {
-            addFunnel(element, funnel);
-        }
-
-        for (final ProcessGroup childGroup : group.getProcessGroups()) {
-            addProcessGroup(element, childGroup, "processGroup");
-        }
-
-        for (final RemoteProcessGroup remoteRef : group.getRemoteProcessGroups()) {
-            addRemoteProcessGroup(element, remoteRef);
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            addConnection(element, connection);
-        }
-    }
-
-    private void addStyle(final Element parentElement, final Map<String, String> style) {
-        final Element element = parentElement.getOwnerDocument().createElement("styles");
-
-        for (final Map.Entry<String, String> entry : style.entrySet()) {
-            final Element styleElement = parentElement.getOwnerDocument().createElement("style");
-            styleElement.setAttribute("name", entry.getKey());
-            styleElement.setTextContent(entry.getValue());
-            element.appendChild(styleElement);
-        }
-
-        parentElement.appendChild(element);
-    }
-
-    private void addLabel(final Element parentElement, final Label label) {
-        final Document doc = parentElement.getOwnerDocument();
-        final Element element = doc.createElement("label");
-        parentElement.appendChild(element);
-        addTextElement(element, "id", label.getIdentifier());
-
-        addPosition(element, label.getPosition());
-        addSize(element, label.getSize());
-        addStyle(element, label.getStyle());
-
-        addTextElement(element, "value", label.getValue());
-        parentElement.appendChild(element);
-    }
-
-    private void addFunnel(final Element parentElement, final Funnel funnel) {
-        final Document doc = parentElement.getOwnerDocument();
-        final Element element = doc.createElement("funnel");
-        parentElement.appendChild(element);
-        addTextElement(element, "id", funnel.getIdentifier());
-        addPosition(element, funnel.getPosition());
-    }
-
-    private void addRemoteProcessGroup(final Element parentElement, final RemoteProcessGroup remoteRef) {
-        final Document doc = parentElement.getOwnerDocument();
-        final Element element = doc.createElement("remoteProcessGroup");
-        parentElement.appendChild(element);
-        addTextElement(element, "id", remoteRef.getIdentifier());
-        addTextElement(element, "name", remoteRef.getName());
-        addPosition(element, remoteRef.getPosition());
-        addTextElement(element, "comment", remoteRef.getComments());
-        addTextElement(element, "url", remoteRef.getTargetUri().toString());
-        addTextElement(element, "timeout", remoteRef.getCommunicationsTimeout());
-        addTextElement(element, "yieldPeriod", remoteRef.getYieldDuration());
-        addTextElement(element, "transmitting", String.valueOf(remoteRef.isTransmitting()));
-
-        for (final RemoteGroupPort port : remoteRef.getInputPorts()) {
-            if (port.hasIncomingConnection()) {
-                addRemoteGroupPort(element, port, "inputPort");
-            }
-        }
-
-        for (final RemoteGroupPort port : remoteRef.getOutputPorts()) {
-            if (!port.getConnections().isEmpty()) {
-                addRemoteGroupPort(element, port, "outputPort");
-            }
-        }
-
-        parentElement.appendChild(element);
-    }
-
-    private void addRemoteGroupPort(final Element parentElement, final RemoteGroupPort port, final String elementName) {
-        final Document doc = parentElement.getOwnerDocument();
-        final Element element = doc.createElement(elementName);
-        parentElement.appendChild(element);
-        addTextElement(element, "id", port.getIdentifier());
-        addTextElement(element, "name", port.getName());
-        addPosition(element, port.getPosition());
-        addTextElement(element, "comments", port.getComments());
-        addTextElement(element, "scheduledState", port.getScheduledState().name());
-        addTextElement(element, "maxConcurrentTasks", port.getMaxConcurrentTasks());
-        addTextElement(element, "useCompression", String.valueOf(((RemoteGroupPort) port).isUseCompression()));
-
-        parentElement.appendChild(element);
-    }
-
-    private void addPort(final Element parentElement, final Port port, final String elementName) {
-        final Document doc = parentElement.getOwnerDocument();
-        final Element element = doc.createElement(elementName);
-        parentElement.appendChild(element);
-        addTextElement(element, "id", port.getIdentifier());
-        addTextElement(element, "name", port.getName());
-        addPosition(element, port.getPosition());
-        addTextElement(element, "comments", port.getComments());
-        addTextElement(element, "scheduledState", port.getScheduledState().name());
-
-        parentElement.appendChild(element);
-    }
-
-    private void addRootGroupPort(final Element parentElement, final RootGroupPort port, final String elementName) {
-        final Document doc = parentElement.getOwnerDocument();
-        final Element element = doc.createElement(elementName);
-        parentElement.appendChild(element);
-        addTextElement(element, "id", port.getIdentifier());
-        addTextElement(element, "name", port.getName());
-        addPosition(element, port.getPosition());
-        addTextElement(element, "comments", port.getComments());
-        addTextElement(element, "scheduledState", port.getScheduledState().name());
-        addTextElement(element, "maxConcurrentTasks", String.valueOf(port.getMaxConcurrentTasks()));
-        for (final String user : port.getUserAccessControl()) {
-            addTextElement(element, "userAccessControl", user);
-        }
-        for (final String group : port.getGroupAccessControl()) {
-            addTextElement(element, "groupAccessControl", group);
-        }
-
-        parentElement.appendChild(element);
-    }
-
-    private void addProcessor(final Element parentElement, final ProcessorNode processor) {
-        final Document doc = parentElement.getOwnerDocument();
-        final Element element = doc.createElement("processor");
-        parentElement.appendChild(element);
-        addTextElement(element, "id", processor.getIdentifier());
-        addTextElement(element, "name", processor.getName());
-
-        addPosition(element, processor.getPosition());
-        addStyle(element, processor.getStyle());
-
-        addTextElement(element, "comment", processor.getComments());
-        addTextElement(element, "class", processor.getProcessor().getClass().getCanonicalName());
-        addTextElement(element, "maxConcurrentTasks", processor.getMaxConcurrentTasks());
-        addTextElement(element, "schedulingPeriod", processor.getSchedulingPeriod());
-        addTextElement(element, "penalizationPeriod", processor.getPenalizationPeriod());
-        addTextElement(element, "yieldPeriod", processor.getYieldPeriod());
-        addTextElement(element, "bulletinLevel", processor.getBulletinLevel().toString());
-        addTextElement(element, "lossTolerant", String.valueOf(processor.isLossTolerant()));
-        addTextElement(element, "scheduledState", processor.getScheduledState().name());
-        addTextElement(element, "schedulingStrategy", processor.getSchedulingStrategy().name());
-        addTextElement(element, "runDurationNanos", processor.getRunDuration(TimeUnit.NANOSECONDS));
-
-        // properties.
-        for (final Map.Entry<PropertyDescriptor, String> entry : processor.getProperties().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-            String value = entry.getValue();
-
-            if (value != null && descriptor.isSensitive()) {
-                value = ENC_PREFIX + encryptor.encrypt(value) + ENC_SUFFIX;
-            }
-
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            final Element propElement = doc.createElement("property");
-            addTextElement(propElement, "name", descriptor.getName());
-            if (value != null) {
-                addTextElement(propElement, "value", value);
-            }
-
-            element.appendChild(propElement);
-        }
-
-        final String annotationData = processor.getAnnotationData();
-        if (annotationData != null) {
-            addTextElement(element, "annotationData", annotationData);
-        }
-
-        for (final Relationship rel : processor.getAutoTerminatedRelationships()) {
-            addTextElement(element, "autoTerminatedRelationship", rel.getName());
-        }
-    }
-
-    private void addConnection(final Element parentElement, final Connection connection) {
-        final Document doc = parentElement.getOwnerDocument();
-        final Element element = doc.createElement("connection");
-        parentElement.appendChild(element);
-        addTextElement(element, "id", connection.getIdentifier());
-        addTextElement(element, "name", connection.getName());
-
-        final Element bendPointsElement = doc.createElement("bendPoints");
-        element.appendChild(bendPointsElement);
-        for (final Position bendPoint : connection.getBendPoints()) {
-            addPosition(bendPointsElement, bendPoint, "bendPoint");
-        }
-
-        addTextElement(element, "labelIndex", connection.getLabelIndex());
-        addTextElement(element, "zIndex", connection.getZIndex());
-
-        final String sourceId = connection.getSource().getIdentifier();
-        final ConnectableType sourceType = connection.getSource().getConnectableType();
-        final String sourceGroupId;
-        if (sourceType == ConnectableType.REMOTE_OUTPUT_PORT) {
-            sourceGroupId = ((RemoteGroupPort) connection.getSource()).getRemoteProcessGroup().getIdentifier();
-        } else {
-            sourceGroupId = connection.getSource().getProcessGroup().getIdentifier();
-        }
-
-        final ConnectableType destinationType = connection.getDestination().getConnectableType();
-        final String destinationId = connection.getDestination().getIdentifier();
-        final String destinationGroupId;
-        if (destinationType == ConnectableType.REMOTE_INPUT_PORT) {
-            destinationGroupId = ((RemoteGroupPort) connection.getDestination()).getRemoteProcessGroup().getIdentifier();
-        } else {
-            destinationGroupId = connection.getDestination().getProcessGroup().getIdentifier();
-        }
-
-        addTextElement(element, "sourceId", sourceId);
-        addTextElement(element, "sourceGroupId", sourceGroupId);
-        addTextElement(element, "sourceType", sourceType.toString());
-
-        addTextElement(element, "destinationId", destinationId);
-        addTextElement(element, "destinationGroupId", destinationGroupId);
-        addTextElement(element, "destinationType", destinationType.toString());
-
-        for (final Relationship relationship : connection.getRelationships()) {
-            addTextElement(element, "relationship", relationship.getName());
-        }
-
-        addTextElement(element, "maxWorkQueueSize", connection.getFlowFileQueue().getBackPressureObjectThreshold());
-        addTextElement(element, "maxWorkQueueDataSize", connection.getFlowFileQueue().getBackPressureDataSizeThreshold());
-
-        addTextElement(element, "flowFileExpiration", connection.getFlowFileQueue().getFlowFileExpiration());
-        for (final FlowFilePrioritizer comparator : connection.getFlowFileQueue().getPriorities()) {
-            final String className = comparator.getClass().getCanonicalName();
-            addTextElement(element, "queuePrioritizerClass", className);
-        }
-
-        parentElement.appendChild(element);
-    }
-
-    private void addTextElement(final Element element, final String name, final long value) {
-        addTextElement(element, name, String.valueOf(value));
-    }
-
-    private void addTextElement(final Element element, final String name, final String value) {
-        final Document doc = element.getOwnerDocument();
-        final Element toAdd = doc.createElement(name);
-        toAdd.setTextContent(value);
-        element.appendChild(toAdd);
-    }
-
-}


[15/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java
deleted file mode 100644
index 72b594a..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java
+++ /dev/null
@@ -1,600 +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.cluster.flow.impl;
-
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import java.util.Arrays;
-import java.util.UUID;
-
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.transform.OutputKeys;
-import javax.xml.transform.Transformer;
-import javax.xml.transform.TransformerFactory;
-import javax.xml.transform.dom.DOMSource;
-import javax.xml.transform.stream.StreamResult;
-
-import org.apache.commons.compress.archivers.ArchiveEntry;
-import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
-import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
-import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
-import org.apache.nifi.cluster.flow.ClusterDataFlow;
-import org.apache.nifi.cluster.flow.DaoException;
-import org.apache.nifi.cluster.flow.DataFlowDao;
-import org.apache.nifi.cluster.flow.PersistedFlowState;
-import org.apache.nifi.cluster.protocol.DataFlow;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.StandardDataFlow;
-import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter;
-import org.apache.nifi.logging.NiFiLog;
-import org.apache.nifi.stream.io.BufferedInputStream;
-import org.apache.nifi.stream.io.BufferedOutputStream;
-import org.apache.nifi.stream.io.ByteArrayInputStream;
-import org.apache.nifi.stream.io.StreamUtils;
-import org.apache.nifi.util.file.FileUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-
-/**
- * Implements the FlowDao interface. The implementation tracks the state of the
- * dataflow by annotating the filename of the flow state file. Specifically, the
- * implementation correlates PersistedFlowState states to filename extensions.
- * The correlation is as follows:
- * <ul>
- * <li> CURRENT maps to flow.xml </li>
- * <li> STALE maps to flow.xml.stale </li>
- * <li> UNKNOWN maps to flow.xml.unknown </li>
- * </ul>
- * Whenever the flow state changes, the flow state file's name is updated to
- * denote its state.
- *
- * The implementation also provides for a restore directory that may be
- * configured for higher availability. At instance creation, if the primary or
- * restore directories have multiple flow state files, an exception is thrown.
- * If the primary directory has a current flow state file, but the restore
- * directory does not, then the primary flow state file is copied to the restore
- * directory. If the restore directory has a current flow state file, but the
- * primary directory does not, then the restore flow state file is copied to the
- * primary directory. If both the primary and restore directories have a current
- * flow state file and the files are different, then an exception is thrown.
- *
- * When the flow state file is saved, it is always saved first to the restore
- * directory followed by a save to the primary directory. When the flow state
- * file is loaded, a check is made to verify that the primary and restore flow
- * state files are both current. If either is not current, then an exception is
- * thrown. The primary flow state file is always read when the load method is
- * called.
- *
- * @author unattributed
- */
-public class DataFlowDaoImpl implements DataFlowDao {
-
-    private final File primaryDirectory;
-    private final File restoreDirectory;
-    private final boolean autoStart;
-    private final String generatedRootGroupId = UUID.randomUUID().toString();
-
-    public static final String STALE_EXT = ".stale";
-    public static final String UNKNOWN_EXT = ".unknown";
-    public static final String FLOW_PACKAGE = "flow.tar";
-    public static final String FLOW_XML_FILENAME = "flow.xml";
-    public static final String TEMPLATES_FILENAME = "templates.xml";
-    public static final String SNIPPETS_FILENAME = "snippets.xml";
-    public static final String CLUSTER_INFO_FILENAME = "cluster-info.xml";
-
-    private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(DataFlowDaoImpl.class));
-
-    public DataFlowDaoImpl(final File primaryDirectory) throws DaoException {
-        this(primaryDirectory, null, false);
-    }
-
-    public DataFlowDaoImpl(final File primaryDirectory, final File restoreDirectory, final boolean autoStart) throws DaoException {
-
-        // sanity check that primary directory is a directory, creating it if necessary
-        if (primaryDirectory == null) {
-            throw new IllegalArgumentException("Primary directory may not be null.");
-        } else if (!primaryDirectory.exists()) {
-            if (!primaryDirectory.mkdir()) {
-                throw new DaoException(String.format("Failed to create primary directory '%s'", primaryDirectory.getAbsolutePath()));
-            }
-        } else if (!primaryDirectory.isDirectory()) {
-            throw new IllegalArgumentException("Primary directory must be a directory.");
-        }
-
-        this.autoStart = autoStart;
-
-        try {
-            this.primaryDirectory = primaryDirectory;
-            this.restoreDirectory = restoreDirectory;
-
-            if (restoreDirectory == null) {
-                // check that we have exactly one current flow state file
-                ensureSingleCurrentStateFile(primaryDirectory);
-            } else {
-
-                // check that restore directory is a directory, creating it if necessary
-                FileUtils.ensureDirectoryExistAndCanAccess(restoreDirectory);
-
-                // check that restore directory is not the same as the primary directory
-                if (primaryDirectory.getAbsolutePath().equals(restoreDirectory.getAbsolutePath())) {
-                    throw new IllegalArgumentException(String.format("Primary directory '%s' is the same as restore directory '%s' ",
-                            primaryDirectory.getAbsolutePath(), restoreDirectory.getAbsolutePath()));
-                }
-
-                final File[] primaryFlowStateFiles = getFlowStateFiles(primaryDirectory);
-                final File[] restoreFlowStateFiles = getFlowStateFiles(restoreDirectory);
-
-                // if more than one state file in either primary or restore, then throw exception
-                if (primaryFlowStateFiles.length > 1) {
-                    throw new IllegalStateException(String.format("Found multiple dataflow state files in primary directory '%s'", primaryDirectory));
-                } else if (restoreFlowStateFiles.length > 1) {
-                    throw new IllegalStateException(String.format("Found multiple dataflow state files in restore directory '%s'", restoreDirectory));
-                }
-
-                // check that the single primary state file we found is current or create a new one
-                final File primaryFlowStateFile = ensureSingleCurrentStateFile(primaryDirectory);
-
-                // check that the single restore state file we found is current or create a new one
-                final File restoreFlowStateFile = ensureSingleCurrentStateFile(restoreDirectory);
-
-                // if there was a difference in flow state file directories, then copy the appropriate files
-                if (restoreFlowStateFiles.length == 0 && primaryFlowStateFiles.length != 0) {
-                    // copy primary state file to restore
-                    FileUtils.copyFile(primaryFlowStateFile, restoreFlowStateFile, false, false, logger);
-                } else if (primaryFlowStateFiles.length == 0 && restoreFlowStateFiles.length != 0) {
-                    // copy restore state file to primary
-                    FileUtils.copyFile(restoreFlowStateFile, primaryFlowStateFile, false, false, logger);
-                } else {
-                    // sync the primary copy with the restore copy
-                    syncWithRestore(primaryFlowStateFile, restoreFlowStateFile);
-                }
-
-            }
-        } catch (final IOException | IllegalArgumentException | IllegalStateException | JAXBException ex) {
-            throw new DaoException(ex);
-        }
-    }
-    
-    
-    private void syncWithRestore(final File primaryFile, final File restoreFile) throws IOException {
-        try (final FileInputStream primaryFis = new FileInputStream(primaryFile);
-             final TarArchiveInputStream primaryIn = new TarArchiveInputStream(primaryFis);
-             final FileInputStream restoreFis = new FileInputStream(restoreFile);
-             final TarArchiveInputStream restoreIn = new TarArchiveInputStream(restoreFis)) {
-            
-            final ArchiveEntry primaryEntry = primaryIn.getNextEntry();
-            final ArchiveEntry restoreEntry = restoreIn.getNextEntry();
-
-            if ( primaryEntry == null && restoreEntry == null ) {
-                return;
-            }
-
-            if ( (primaryEntry == null && restoreEntry != null) || (primaryEntry != null && restoreEntry == null) ) {
-                throw new IllegalStateException(String.format("Primary file '%s' is different than restore file '%s'",
-                        primaryFile.getAbsoluteFile(), restoreFile.getAbsolutePath()));
-            }
-            
-            final byte[] primaryMd5 = calculateMd5(primaryIn);
-            final byte[] restoreMd5 = calculateMd5(restoreIn);
-            
-            if ( !Arrays.equals(primaryMd5, restoreMd5) ) {
-                throw new IllegalStateException(String.format("Primary file '%s' is different than restore file '%s'",
-                        primaryFile.getAbsoluteFile(), restoreFile.getAbsolutePath()));
-            }
-        }
-    }
-    
-    private byte[] calculateMd5(final InputStream in) throws IOException {
-        final MessageDigest digest;
-        try {
-            digest = MessageDigest.getInstance("MD5");
-        } catch (final NoSuchAlgorithmException nsae) {
-            throw new IOException(nsae);
-        }
-        
-        int len;
-        final byte[] buffer = new byte[8192];
-        while ((len = in.read(buffer)) > -1) {
-            if (len > 0) {
-                digest.update(buffer, 0, len);
-            }
-        }
-        return digest.digest();
-    }
-
-    @Override
-    public ClusterDataFlow loadDataFlow() throws DaoException {
-        try {
-            return parseDataFlow(getExistingFlowStateFile(primaryDirectory));
-        } catch (final IOException | JAXBException ex) {
-            throw new DaoException(ex);
-        }
-    }
-
-    @Override
-    public void saveDataFlow(final ClusterDataFlow dataFlow) throws DaoException {
-        try {
-
-            final File primaryStateFile = getFlowStateFile(primaryDirectory);
-
-            // write to restore before writing to primary in case primary experiences problems
-            if (restoreDirectory != null) {
-                final File restoreStateFile = getFlowStateFile(restoreDirectory);
-                if (restoreStateFile == null) {
-                    if (primaryStateFile == null) {
-                        writeDataFlow(createNewFlowStateFile(restoreDirectory), dataFlow);
-                    } else {
-                        throw new DaoException(String.format("Unable to save dataflow because dataflow state file in primary directory '%s' exists, but it does not exist in the restore directory '%s'",
-                                primaryDirectory.getAbsolutePath(), restoreDirectory.getAbsolutePath()));
-                    }
-                } else {
-                    if (primaryStateFile == null) {
-                        throw new DaoException(String.format("Unable to save dataflow because dataflow state file in restore directory '%s' exists, but it does not exist in the primary directory '%s'",
-                                restoreDirectory.getAbsolutePath(), primaryDirectory.getAbsolutePath()));
-                    } else {
-                        final PersistedFlowState primaryFlowState = getPersistedFlowState(primaryStateFile);
-                        final PersistedFlowState restoreFlowState = getPersistedFlowState(restoreStateFile);
-                        if (primaryFlowState == restoreFlowState) {
-                            writeDataFlow(restoreStateFile, dataFlow);
-                        } else {
-                            throw new DaoException(String.format("Unable to save dataflow because state file in primary directory '%s' has state '%s', but the state file in the restore directory '%s' has state '%s'",
-                                    primaryDirectory.getAbsolutePath(), primaryFlowState, restoreDirectory.getAbsolutePath(), restoreFlowState));
-                        }
-                    }
-                }
-            }
-
-            // write dataflow to primary 
-            if (primaryStateFile == null) {
-                writeDataFlow(createNewFlowStateFile(primaryDirectory), dataFlow);
-            } else {
-                writeDataFlow(primaryStateFile, dataFlow);
-            }
-
-        } catch (final IOException | JAXBException ex) {
-            throw new DaoException(ex);
-        }
-    }
-
-    @Override
-    public PersistedFlowState getPersistedFlowState() {
-        // trust restore over primary if configured for restore
-        if (restoreDirectory == null) {
-            return getPersistedFlowState(getExistingFlowStateFile(primaryDirectory));
-        } else {
-            return getPersistedFlowState(getExistingFlowStateFile(restoreDirectory));
-        }
-    }
-
-    @Override
-    public void setPersistedFlowState(final PersistedFlowState flowState) throws DaoException {
-        // rename restore before primary if configured for restore
-        if (restoreDirectory != null) {
-            renameFlowStateFile(getExistingFlowStateFile(restoreDirectory), flowState);
-        }
-        renameFlowStateFile(getExistingFlowStateFile(primaryDirectory), flowState);
-    }
-
-    private File ensureSingleCurrentStateFile(final File dir) throws IOException, JAXBException {
-
-        // ensure that we have at most one state file and if we have one, it is current
-        final File[] directoryFlowStateFiles = getFlowStateFiles(dir);
-        if (directoryFlowStateFiles.length > 1) {
-            throw new DaoException(String.format("Found multiple dataflow state files in directory '%s'", dir));
-        } else if (directoryFlowStateFiles.length == 0) {
-            // create a new file if none exist
-            return createNewFlowStateFile(dir);
-        } else {
-            // check that the single flow state file is current
-            final PersistedFlowState flowState = getPersistedFlowState(directoryFlowStateFiles[0]);
-            if (PersistedFlowState.CURRENT == flowState) {
-                return directoryFlowStateFiles[0];
-            } else {
-                throw new DaoException(String.format("Dataflow state file '%s' must be current.", directoryFlowStateFiles[0].getAbsolutePath()));
-            }
-        }
-
-    }
-
-    private PersistedFlowState getPersistedFlowState(final File file) {
-        final String path = file.getAbsolutePath();
-        if (path.endsWith(STALE_EXT)) {
-            return PersistedFlowState.STALE;
-        } else if (path.endsWith(UNKNOWN_EXT)) {
-            return PersistedFlowState.UNKNOWN;
-        } else {
-            return PersistedFlowState.CURRENT;
-        }
-    }
-
-    private File getFlowStateFile(final File dir) {
-        final File[] files = getFlowStateFiles(dir);
-        if (files.length > 1) {
-            throw new IllegalStateException(String.format("Expected at most one dataflow state file, but found %s files.", files.length));
-        } else if (files.length == 0) {
-            return null;
-        } else {
-            return files[0];
-        }
-    }
-
-    private File getExistingFlowStateFile(final File dir) {
-        final File file = getFlowStateFile(dir);
-        if (file == null) {
-            throw new IllegalStateException(String.format("Expected a dataflow state file, but none existed in directory '%s'", dir.getAbsolutePath()));
-        }
-        return file;
-    }
-
-    private File[] getFlowStateFiles(final File dir) {
-        final File[] files = dir.listFiles(new FilenameFilter() {
-            @Override
-            public boolean accept(File dir, String name) {
-                return (name.equals(FLOW_PACKAGE) || name.endsWith(STALE_EXT) || name.endsWith(UNKNOWN_EXT));
-            }
-        });
-
-        if (files == null) {
-            return new File[0];
-        } else {
-            return files;
-        }
-    }
-
-    private File removeStateFileExtension(final File file) {
-
-        final String path = file.getAbsolutePath();
-        final int stateFileExtIndex;
-        if (path.endsWith(STALE_EXT)) {
-            stateFileExtIndex = path.lastIndexOf(STALE_EXT);
-        } else if (path.endsWith(UNKNOWN_EXT)) {
-            stateFileExtIndex = path.lastIndexOf(UNKNOWN_EXT);
-        } else {
-            stateFileExtIndex = path.length();
-        }
-
-        return new File(path.substring(0, stateFileExtIndex));
-    }
-
-    private File addStateFileExtension(final File file, final PersistedFlowState state) {
-        switch (state) {
-            case CURRENT: {
-                return file;
-            }
-            case STALE: {
-                return new File(file.getAbsolutePath() + STALE_EXT);
-            }
-            case UNKNOWN: {
-                return new File(file.getAbsolutePath() + UNKNOWN_EXT);
-            }
-            default: {
-                throw new RuntimeException("Unsupported PersistedFlowState Enum value: " + state);
-            }
-        }
-    }
-
-    private File createNewFlowStateFile(final File dir) throws IOException, JAXBException {
-        final File stateFile = new File(dir, FLOW_PACKAGE);
-        stateFile.createNewFile();
-
-        final byte[] flowBytes = getEmptyFlowBytes();
-        final byte[] templateBytes = new byte[0];
-        final byte[] snippetBytes = new byte[0];
-        final DataFlow dataFlow = new StandardDataFlow(flowBytes, templateBytes, snippetBytes);
-
-        final ClusterMetadata clusterMetadata = new ClusterMetadata();
-        writeDataFlow(stateFile, dataFlow, clusterMetadata);
-
-        return stateFile;
-    }
-
-    private byte[] getEmptyFlowBytes() throws IOException {
-        try {
-            final DocumentBuilder docBuilder = DocumentBuilderFactory.newInstance().newDocumentBuilder();
-            final Document document = docBuilder.newDocument();
-
-            final Element controller = document.createElement("flowController");
-            document.appendChild(controller);
-
-            controller.appendChild(createTextElement(document, "maxThreadCount", "15"));
-
-            final Element rootGroup = document.createElement("rootGroup");
-            rootGroup.appendChild(createTextElement(document, "id", generatedRootGroupId));
-            rootGroup.appendChild(createTextElement(document, "name", "NiFi Flow"));
-
-            // create the position element
-            final Element positionElement = createTextElement(document, "position", "");
-            positionElement.setAttribute("x", "0.0");
-            positionElement.setAttribute("y", "0.0");
-            rootGroup.appendChild(positionElement);
-
-            rootGroup.appendChild(createTextElement(document, "comment", ""));
-            controller.appendChild(rootGroup);
-
-            final Transformer transformer = TransformerFactory.newInstance().newTransformer();
-            transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "2");
-            transformer.setOutputProperty(OutputKeys.INDENT, "yes");
-
-            final DOMSource source = new DOMSource(document);
-            final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-            final StreamResult result = new StreamResult(baos);
-            transformer.transform(source, result);
-
-            return baos.toByteArray();
-        } catch (final Exception e) {
-            throw new IOException(e);
-        }
-    }
-
-    private Element createTextElement(final Document document, final String elementName, final String value) {
-        final Element element = document.createElement(elementName);
-        element.setTextContent(value);
-        return element;
-    }
-
-    private void renameFlowStateFile(final File flowStateFile, final PersistedFlowState newState) throws DaoException {
-        final PersistedFlowState existingState = getPersistedFlowState(flowStateFile);
-        if (existingState != newState) {
-            final File newFlowStateFile = addStateFileExtension(removeStateFileExtension(flowStateFile), newState);
-            if (flowStateFile.renameTo(newFlowStateFile) == false) {
-                throw new DaoException(
-                        String.format("Failed to rename flow state file '%s' to new name '%s'", flowStateFile.getAbsolutePath(), newFlowStateFile.getAbsolutePath()));
-            }
-        }
-    }
-
-    private ClusterDataFlow parseDataFlow(final File file) throws IOException, JAXBException, DaoException {
-        byte[] flowBytes = new byte[0];
-        byte[] templateBytes = new byte[0];
-        byte[] snippetBytes = new byte[0];
-        byte[] clusterInfoBytes = new byte[0];
-
-        try (final InputStream inStream = new FileInputStream(file);
-                final TarArchiveInputStream tarIn = new TarArchiveInputStream(new BufferedInputStream(inStream))) {
-            TarArchiveEntry tarEntry;
-            while ((tarEntry = tarIn.getNextTarEntry()) != null) {
-                switch (tarEntry.getName()) {
-                    case FLOW_XML_FILENAME:
-                        flowBytes = new byte[(int) tarEntry.getSize()];
-                        StreamUtils.fillBuffer(tarIn, flowBytes, true);
-                        break;
-                    case TEMPLATES_FILENAME:
-                        templateBytes = new byte[(int) tarEntry.getSize()];
-                        StreamUtils.fillBuffer(tarIn, templateBytes, true);
-                        break;
-                    case SNIPPETS_FILENAME:
-                        snippetBytes = new byte[(int) tarEntry.getSize()];
-                        StreamUtils.fillBuffer(tarIn, snippetBytes, true);
-                        break;
-                    case CLUSTER_INFO_FILENAME:
-                        clusterInfoBytes = new byte[(int) tarEntry.getSize()];
-                        StreamUtils.fillBuffer(tarIn, clusterInfoBytes, true);
-                        break;
-                    default:
-                        throw new DaoException("Found Unexpected file in dataflow configuration: " + tarEntry.getName());
-                }
-            }
-        }
-
-        final ClusterMetadata clusterMetadata;
-        if (clusterInfoBytes.length == 0) {
-            clusterMetadata = null;
-        } else {
-            final Unmarshaller clusterMetadataUnmarshaller = ClusterMetadata.jaxbCtx.createUnmarshaller();
-            clusterMetadata = (ClusterMetadata) clusterMetadataUnmarshaller.unmarshal(new ByteArrayInputStream(clusterInfoBytes));
-        }
-
-        final StandardDataFlow dataFlow = new StandardDataFlow(flowBytes, templateBytes, snippetBytes);
-        dataFlow.setAutoStartProcessors(autoStart);
-
-        return new ClusterDataFlow(dataFlow, (clusterMetadata == null) ? null : clusterMetadata.getPrimaryNodeId());
-    }
-
-    private void writeDataFlow(final File file, final ClusterDataFlow clusterDataFlow) throws IOException, JAXBException {
-
-        // get the data flow
-        DataFlow dataFlow = clusterDataFlow.getDataFlow();
-
-        // if no dataflow, then write a new dataflow
-        if (dataFlow == null) {
-            dataFlow = new StandardDataFlow(new byte[0], new byte[0], new byte[0]);
-        }
-
-        // setup the cluster metadata
-        final ClusterMetadata clusterMetadata = new ClusterMetadata();
-        clusterMetadata.setPrimaryNodeId(clusterDataFlow.getPrimaryNodeId());
-
-        // write to disk
-        writeDataFlow(file, dataFlow, clusterMetadata);
-    }
-
-    private void writeTarEntry(final TarArchiveOutputStream tarOut, final String filename, final byte[] bytes) throws IOException {
-        final TarArchiveEntry flowEntry = new TarArchiveEntry(filename);
-        flowEntry.setSize(bytes.length);
-        tarOut.putArchiveEntry(flowEntry);
-        tarOut.write(bytes);
-        tarOut.closeArchiveEntry();
-    }
-
-    private void writeDataFlow(final File file, final DataFlow dataFlow, final ClusterMetadata clusterMetadata) throws IOException, JAXBException {
-
-        try (final OutputStream fos = new FileOutputStream(file);
-                final TarArchiveOutputStream tarOut = new TarArchiveOutputStream(new BufferedOutputStream(fos))) {
-
-            writeTarEntry(tarOut, FLOW_XML_FILENAME, dataFlow.getFlow());
-            writeTarEntry(tarOut, TEMPLATES_FILENAME, dataFlow.getTemplates());
-            writeTarEntry(tarOut, SNIPPETS_FILENAME, dataFlow.getSnippets());
-
-            final ByteArrayOutputStream baos = new ByteArrayOutputStream(256);
-            writeClusterMetadata(clusterMetadata, baos);
-            final byte[] clusterInfoBytes = baos.toByteArray();
-
-            writeTarEntry(tarOut, CLUSTER_INFO_FILENAME, clusterInfoBytes);
-        }
-    }
-
-    private void writeClusterMetadata(final ClusterMetadata clusterMetadata, final OutputStream os) throws IOException, JAXBException {
-        // write cluster metadata to output stream
-        final Marshaller marshaller = ClusterMetadata.jaxbCtx.createMarshaller();
-        marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
-        marshaller.setProperty(Marshaller.JAXB_FRAGMENT, true);
-        marshaller.setProperty(Marshaller.JAXB_ENCODING, "UTF-8");
-        marshaller.marshal(clusterMetadata, os);
-    }
-
-    @XmlRootElement(name = "clusterMetadata")
-    private static class ClusterMetadata {
-
-        private NodeIdentifier primaryNodeId;
-
-        private static final JAXBContext jaxbCtx;
-
-        static {
-            try {
-                jaxbCtx = JAXBContext.newInstance(ClusterMetadata.class);
-            } catch (final JAXBException je) {
-                throw new RuntimeException(je);
-            }
-        }
-
-        @XmlJavaTypeAdapter(NodeIdentifierAdapter.class)
-        public NodeIdentifier getPrimaryNodeId() {
-            return primaryNodeId;
-        }
-
-        public void setPrimaryNodeId(final NodeIdentifier primaryNodeId) {
-            this.primaryNodeId = primaryNodeId;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java
deleted file mode 100644
index e135af3..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java
+++ /dev/null
@@ -1,356 +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.cluster.flow.impl;
-
-import java.util.Collections;
-import java.util.Date;
-import java.util.Set;
-import java.util.Timer;
-import java.util.TimerTask;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-import org.apache.nifi.cluster.flow.ClusterDataFlow;
-import org.apache.nifi.cluster.flow.DaoException;
-import org.apache.nifi.cluster.flow.DataFlowDao;
-import org.apache.nifi.cluster.flow.DataFlowManagementService;
-import org.apache.nifi.cluster.flow.PersistedFlowState;
-import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.StandardDataFlow;
-import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
-import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
-import org.apache.nifi.logging.NiFiLog;
-import org.apache.nifi.util.FormatUtils;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Implements FlowManagementService interface. The service tries to keep the
- * cluster's flow current with regards to the available nodes.
- *
- * The instance may be configured with a retrieval delay, which will reduce the
- * number of retrievals performed by the service at the expense of increasing
- * the chances that the service will not be able to provide a current flow to
- * the caller.
- *
- * By default, the service will try to update the flow as quickly as possible.
- * Configuring a delay enables a less aggressive retrieval strategy.
- * Specifically, the eligible retrieval time is reset every time the flow state
- * is set to STALE. If the state is set to UNKNOWN or CURRENT, then the flow
- * will not be retrieved.
- *
- * @author unattributed
- */
-public class DataFlowManagementServiceImpl implements DataFlowManagementService {
-
-    /*
-     * Developer Note: 
-     * 
-     * This class maintains an ExecutorService and a Runnable.
-     * Although the class is not externally threadsafe, its internals are protected to
-     * accommodate multithread access between the ExecutorServer and the Runnable.
-     * 
-     */
-    private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(DataFlowManagementServiceImpl.class));
-
-    private final DataFlowDao flowDao;
-
-    private final ClusterManagerProtocolSender sender;
-
-    private final Set<NodeIdentifier> nodeIds = new CopyOnWriteArraySet<>();
-
-    private final AtomicBoolean stopRequested = new AtomicBoolean(false);
-
-    private final AtomicLong lastRetrievalTime = new AtomicLong(-1);
-
-    private Timer flowRetriever;
-
-    private long retrievableAfterTime = 0L;
-
-    private AtomicInteger retrievalDelaySeconds = new AtomicInteger(0);
-
-    private final TimingReentrantLock resourceLock = new TimingReentrantLock(new ReentrantLock());
-
-    public DataFlowManagementServiceImpl(final DataFlowDao flowDao, final ClusterManagerProtocolSender sender) {
-        if (flowDao == null) {
-            throw new IllegalArgumentException("Flow DAO may not be null.");
-        } else if (sender == null) {
-            throw new IllegalArgumentException("Cluster Manager Protocol Sender may not be null.");
-        }
-        this.flowDao = flowDao;
-        this.sender = sender;
-    }
-
-    @Override
-    public void start() {
-
-        if (isRunning()) {
-            throw new IllegalArgumentException("Instance is already running.");
-        }
-
-        // reset stop requested
-        stopRequested.set(false);
-
-        // setup flow retreiver timer
-        flowRetriever = new Timer("Flow Management Service", /* is daemon */ true);
-        flowRetriever.schedule(new FlowRetrieverTimerTask(), 0, 500);
-    }
-
-    @Override
-    public boolean isRunning() {
-        return (flowRetriever != null);
-    }
-
-    @Override
-    public void stop() {
-
-        if (isRunning() == false) {
-            throw new IllegalArgumentException("Instance is already stopped.");
-        }
-
-        // record stop request
-        stopRequested.set(true);
-
-        flowRetriever.cancel();
-        flowRetriever = null;
-
-    }
-
-    @Override
-    public ClusterDataFlow loadDataFlow() throws DaoException {
-        resourceLock.lock();
-        try {
-            return flowDao.loadDataFlow();
-        } finally {
-            resourceLock.unlock("loadDataFlow");
-        }
-    }
-
-    @Override
-    public void updatePrimaryNode(final NodeIdentifier nodeId) {
-        resourceLock.lock();
-        try {
-            final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow();
-
-            final StandardDataFlow dataFlow;
-            if (existingClusterDataFlow == null) {
-                dataFlow = null;
-            } else {
-                dataFlow = existingClusterDataFlow.getDataFlow();
-            }
-
-            flowDao.saveDataFlow(new ClusterDataFlow(dataFlow, nodeId));
-        } finally {
-            resourceLock.unlock("updatePrimaryNode");
-        }
-    }
-
-    @Override
-    public PersistedFlowState getPersistedFlowState() {
-        resourceLock.lock();
-        try {
-            return flowDao.getPersistedFlowState();
-        } finally {
-            resourceLock.unlock("getPersistedFlowState");
-        }
-    }
-
-    @Override
-    public boolean isFlowCurrent() {
-        return PersistedFlowState.CURRENT == getPersistedFlowState();
-    }
-
-    @Override
-    public void setPersistedFlowState(final PersistedFlowState flowState) {
-        // lock to ensure state change and retrievable time update are atomic
-        resourceLock.lock();
-        try {
-            flowDao.setPersistedFlowState(flowState);
-            if (PersistedFlowState.STALE == flowState) {
-                retrievableAfterTime = new Date().getTime() + (getRetrievalDelaySeconds() * 1000);
-            } else if (PersistedFlowState.UNKNOWN == flowState || PersistedFlowState.CURRENT == flowState) {
-                retrievableAfterTime = Long.MAX_VALUE;
-            }
-        } finally {
-            resourceLock.unlock("setPersistedFlowState");
-        }
-    }
-
-    @Override
-    public Set<NodeIdentifier> getNodeIds() {
-        return Collections.unmodifiableSet(nodeIds);
-    }
-
-    @Override
-    public void setNodeIds(final Set<NodeIdentifier> nodeIds) {
-
-        if (nodeIds == null) {
-            throw new IllegalArgumentException("Node IDs may not be null.");
-        }
-
-        resourceLock.lock();
-        try {
-
-            if (this.nodeIds.equals(nodeIds)) {
-                return;
-            }
-
-            this.nodeIds.clear();
-            this.nodeIds.addAll(nodeIds);
-
-        } finally {
-            resourceLock.unlock("setNodeIds");
-        }
-
-    }
-
-    @Override
-    public int getRetrievalDelaySeconds() {
-        return retrievalDelaySeconds.get();
-    }
-
-    @Override
-    public void setRetrievalDelay(final String retrievalDelay) {
-        this.retrievalDelaySeconds.set((int) FormatUtils.getTimeDuration(retrievalDelay, TimeUnit.SECONDS));
-    }
-
-    public ClusterManagerProtocolSender getSender() {
-        return sender;
-    }
-
-    public long getLastRetrievalTime() {
-        return lastRetrievalTime.get();
-    }
-
-    /**
-     * A timer task for issuing FlowRequestMessage messages to nodes to retrieve
-     * an updated flow.
-     */
-    private class FlowRetrieverTimerTask extends TimerTask {
-
-        @Override
-        public void run() {
-
-            resourceLock.lock();
-            try {
-                // if flow is current, then we're done
-                if (isFlowCurrent()) {
-                    return;
-                }
-            } catch (final Exception ex) {
-                logger.info("Encountered exception checking if flow is current caused by " + ex, ex);
-            } finally {
-                resourceLock.unlock("FlowRetrieverTimerTask - isFlowCurrent");
-            }
-
-            final FlowRequestMessage request = new FlowRequestMessage();
-            for (final NodeIdentifier nodeId : getNodeIds()) {
-                try {
-                    // setup request
-                    request.setNodeId(nodeId);
-
-                    // record request time
-                    final long requestSentTime = new Date().getTime();
-
-                    resourceLock.lock();
-                    try {
-                        // sanity checks before making request
-                        if (stopRequested.get()) {  // did we receive a stop request
-                            logger.debug("Stopping runnable prematurely because a request to stop was issued.");
-                            return;
-                        } else if (requestSentTime < retrievableAfterTime) {
-                            /*
-                             * Retrievable after time was updated while obtaining
-                             * the lock, so try again later
-                             */
-                            return;
-                        }
-                    } finally {
-                        resourceLock.unlock("FlowRetrieverTimerTask - check stopRequested");
-                    }
-
-                    // send request
-                    final FlowResponseMessage response = sender.requestFlow(request);
-
-                    resourceLock.lock();
-                    try {
-                        // check if the retrieved flow is still valid
-                        if (requestSentTime > retrievableAfterTime) {
-                            logger.info("Saving retrieved flow.");
-
-                            final StandardDataFlow dataFlow = response.getDataFlow();
-                            final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow();
-                            final ClusterDataFlow currentClusterDataFlow;
-                            if (existingClusterDataFlow == null) {
-                                currentClusterDataFlow = new ClusterDataFlow(dataFlow, null);
-                            } else {
-                                currentClusterDataFlow = new ClusterDataFlow(dataFlow, existingClusterDataFlow.getPrimaryNodeId());
-                            }
-                            flowDao.saveDataFlow(currentClusterDataFlow);
-                            flowDao.setPersistedFlowState(PersistedFlowState.CURRENT);
-                            lastRetrievalTime.set(new Date().getTime());
-                        }
-
-                        /*
-                         * Retrievable after time was updated while requesting
-                         * the flow, so try again later.
-                         */
-                    } finally {
-                        resourceLock.unlock("FlowRetrieverTimerTask - saveDataFlow");
-                    }
-
-                } catch (final Throwable t) {
-                    logger.info("Encountered exception retrieving flow from node " + nodeId + " caused by " + t, t);
-                }
-            }
-        }
-    }
-
-    private static class TimingReentrantLock {
-
-        private final Lock lock;
-        private static final Logger logger = LoggerFactory.getLogger("dataFlowManagementService.lock");
-
-        private final ThreadLocal<Long> lockTime = new ThreadLocal<>();
-
-        public TimingReentrantLock(final Lock lock) {
-            this.lock = lock;
-        }
-
-        public void lock() {
-            lock.lock();
-            lockTime.set(System.nanoTime());
-        }
-
-        public void unlock(final String task) {
-            final long nanosLocked = System.nanoTime() - lockTime.get();
-            lock.unlock();
-
-            final long millisLocked = TimeUnit.MILLISECONDS.convert(nanosLocked, TimeUnit.NANOSECONDS);
-            if (millisLocked > 100L) {
-                logger.debug("Lock held for {} milliseconds for task: {}", millisLocked, task);
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java
deleted file mode 100644
index 0fcac8c..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java
+++ /dev/null
@@ -1,225 +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.cluster.manager;
-
-import org.apache.nifi.cluster.manager.exception.PrimaryRoleAssignmentException;
-import org.apache.nifi.cluster.manager.exception.IllegalNodeDisconnectionException;
-import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
-import org.apache.nifi.cluster.manager.exception.IneligiblePrimaryNodeException;
-import org.apache.nifi.cluster.manager.exception.IllegalNodeDeletionException;
-import org.apache.nifi.cluster.manager.exception.IllegalNodeReconnectionException;
-import org.apache.nifi.cluster.manager.exception.NodeDisconnectionException;
-import org.apache.nifi.cluster.NodeInformant;
-import org.apache.nifi.cluster.event.Event;
-import org.apache.nifi.cluster.node.Node;
-import org.apache.nifi.cluster.node.Node.Status;
-import org.apache.nifi.cluster.protocol.ConnectionRequest;
-import org.apache.nifi.cluster.protocol.ConnectionResponse;
-import org.apache.nifi.cluster.protocol.Heartbeat;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.controller.status.ProcessGroupStatus;
-import org.apache.nifi.diagnostics.SystemDiagnostics;
-import org.apache.nifi.reporting.BulletinRepository;
-
-import java.util.List;
-import java.util.Set;
-
-/**
- * Defines the interface for a ClusterManager. The cluster manager is a
- * threadsafe centralized manager for a cluster. Members of a cluster are nodes.
- * A member becomes a node by issuing a connection request to the manager. The
- * manager maintains the set of nodes. Nodes may be disconnected, reconnected,
- * and deleted.
- *
- * Nodes are responsible for sending heartbeats to the manager to indicate their
- * liveliness. A manager may disconnect a node if it does not receive a
- * heartbeat within a configurable time period. A cluster manager instance may
- * be configured with how often to monitor received heartbeats
- * (getHeartbeatMonitoringIntervalSeconds()) and the maximum time that may
- * elapse between node heartbeats before disconnecting the node
- * (getMaxHeartbeatGapSeconds()).
- *
- * Since only a single node may execute isolated processors, the cluster manager
- * maintains the notion of a primary node. The primary node is chosen at cluster
- * startup and retains the role until a user requests a different node to be the
- * primary node.
- *
- * @author unattributed
- */
-public interface ClusterManager extends NodeInformant {
-
-    /**
-     * Handles a node's heartbeat.
-     *
-     * @param heartbeat a heartbeat
-     *
-     */
-    void handleHeartbeat(Heartbeat heartbeat);
-
-    /**
-     * @param statuses the statuses of the nodes
-     * @return the set of nodes
-     */
-    Set<Node> getNodes(Status... statuses);
-
-    /**
-     * @param nodeId
-     * @return returns the node with the given identifier or null if node does
-     * not exist
-     */
-    Node getNode(String nodeId);
-
-    /**
-     * @param statuses
-     * @return the set of node identifiers with the given node status
-     */
-    Set<NodeIdentifier> getNodeIds(Status... statuses);
-
-    /**
-     * Deletes the node with the given node identifier. If the given node is the
-     * primary node, then a subsequent request may be made to the manager to set
-     * a new primary node.
-     *
-     * @param nodeId the node identifier
-     * @param userDn the Distinguished Name of the user requesting the node be
-     * deleted from the cluster
-     *
-     * @throws UnknownNodeException if the node does not exist
-     * @throws IllegalNodeDeletionException if the node is not in a disconnected
-     * state
-     */
-    void deleteNode(String nodeId, String userDn) throws UnknownNodeException, IllegalNodeDeletionException;
-
-    /**
-     * Requests a connection to the cluster.
-     *
-     * @param request the request
-     *
-     * @return the response
-     */
-    ConnectionResponse requestConnection(ConnectionRequest request);
-
-    /**
-     * Services reconnection requests for a given node. If the node indicates
-     * reconnection failure, then the node will be set to disconnected.
-     * Otherwise, a reconnection request will be sent to the node, initiating
-     * the connection handshake.
-     *
-     * @param nodeId a node identifier
-     * @param userDn the Distinguished Name of the user requesting the
-     * reconnection
-     *
-     * @throws UnknownNodeException if the node does not exist
-     * @throws IllegalNodeReconnectionException if the node is not disconnected
-     */
-    void requestReconnection(String nodeId, String userDn) throws UnknownNodeException, IllegalNodeReconnectionException;
-
-    /**
-     * Requests the node with the given identifier be disconnected.
-     *
-     * @param nodeId the node identifier
-     * @param userDn the Distinguished Name of the user requesting the
-     * disconnection
-     *
-     * @throws UnknownNodeException if the node does not exist
-     * @throws IllegalNodeDisconnectionException if the node cannot be
-     * disconnected due to the cluster's state (e.g., node is last connected
-     * node or node is primary)
-     * @throws UnknownNodeException if the node does not exist
-     * @throws IllegalNodeDisconnectionException if the node is not disconnected
-     * @throws NodeDisconnectionException if the disconnection failed
-     */
-    void requestDisconnection(String nodeId, String userDn) throws UnknownNodeException, IllegalNodeDisconnectionException, NodeDisconnectionException;
-
-    /**
-     * @return the time in seconds to wait between successive executions of
-     * heartbeat monitoring
-     */
-    int getHeartbeatMonitoringIntervalSeconds();
-
-    /**
-     * @return the maximum time in seconds that is allowed between successive
-     * heartbeats of a node before disconnecting the node
-     */
-    int getMaxHeartbeatGapSeconds();
-
-    /**
-     * Returns a list of node events for the node with the given identifier. The
-     * events will be returned in order of most recent to least recent according
-     * to the creation date of the event.
-     *
-     * @param nodeId the node identifier
-     *
-     * @return the list of events or an empty list if no node exists with the
-     * given identifier
-     */
-    List<Event> getNodeEvents(final String nodeId);
-
-    /**
-     * Revokes the primary role from the current primary node and assigns the
-     * primary role to given given node ID.
-     *
-     * If role revocation fails, then the current primary node is set to
-     * disconnected while retaining the primary role and no role assignment is
-     * performed.
-     *
-     * If role assignment fails, then the given node is set to disconnected and
-     * is given the primary role.
-     *
-     * @param nodeId the node identifier
-     * @param userDn the Distinguished Name of the user requesting that the
-     * Primary Node be assigned
-     *
-     * @throws UnknownNodeException if the node with the given identifier does
-     * not exist
-     * @throws IneligiblePrimaryNodeException if the node with the given
-     * identifier is not eligible to be the primary node
-     * @throws PrimaryRoleAssignmentException if the cluster was unable to
-     * change the primary role to the requested node
-     */
-    void setPrimaryNode(String nodeId, String userDn) throws UnknownNodeException, IneligiblePrimaryNodeException, PrimaryRoleAssignmentException;
-
-    /**
-     * @return the primary node of the cluster or null if no primary node exists
-     */
-    Node getPrimaryNode();
-
-    /**
-     * Returns the bulletin repository.
-     *
-     * @return
-     */
-    BulletinRepository getBulletinRepository();
-
-    /**
-     * Returns a {@link ProcessGroupStatus} that represents the status of all
-     * nodes with the given {@link Status}es for the given ProcessGroup id, or
-     * null if no nodes exist with the given statuses
-     *
-     * @param groupId
-     * @return
-     */
-    ProcessGroupStatus getProcessGroupStatus(String groupId);
-
-    /**
-     * Returns a merged representation of the System Diagnostics for all nodes
-     * in the cluster
-     *
-     * @return
-     */
-    SystemDiagnostics getSystemDiagnostics();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpClusterManager.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpClusterManager.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpClusterManager.java
deleted file mode 100644
index 2cf5812..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpClusterManager.java
+++ /dev/null
@@ -1,169 +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.cluster.manager;
-
-import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException;
-import org.apache.nifi.cluster.manager.exception.UriConstructionException;
-import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException;
-import org.apache.nifi.cluster.manager.exception.NoConnectedNodesException;
-import org.apache.nifi.cluster.manager.exception.NoResponseFromNodesException;
-import org.apache.nifi.cluster.manager.exception.SafeModeMutableRequestException;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-
-import java.net.URI;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Extends the ClusterManager interface to define how requests issued to the
- * cluster manager are federated to the nodes. Specifically, the HTTP protocol
- * is used for communicating requests to the cluster manager and to the nodes.
- *
- * @author unattributed
- */
-public interface HttpClusterManager extends ClusterManager {
-
-    /**
-     * Federates the HTTP request to all connected nodes in the cluster. The
-     * given URI's host and port will not be used and instead will be adjusted
-     * for each node's host and port. The node URIs are guaranteed to be
-     * constructed before issuing any requests, so if a UriConstructionException
-     * is thrown, then it is guaranteed that no request was issued.
-     *
-     * @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD)
-     * @param uri the base request URI (up to, but not including, the query
-     * string)
-     * @param parameters the request parameters
-     * @param headers the request headers
-     *
-     * @return the client response
-     *
-     * @throws NoConnectedNodesException if no nodes are connected as results of
-     * the request
-     * @throws NoResponseFromNodesException if no response could be obtained
-     * @throws UriConstructionException if there was an issue constructing the
-     * URIs tailored for each individual node
-     * @throws ConnectingNodeMutableRequestException if the request was a PUT,
-     * POST, DELETE and a node is connecting to the cluster
-     * @throws DisconnectedNodeMutableRequestException if the request was a PUT,
-     * POST, DELETE and a node is disconnected from the cluster
-     * @throws SafeModeMutableRequestException if the request was a PUT, POST,
-     * DELETE and a the cluster is in safe mode
-     */
-    NodeResponse applyRequest(String method, URI uri, Map<String, List<String>> parameters, Map<String, String> headers)
-            throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException,
-            DisconnectedNodeMutableRequestException, SafeModeMutableRequestException;
-
-    /**
-     * Federates the HTTP request to the nodes specified. The given URI's host
-     * and port will not be used and instead will be adjusted for each node's
-     * host and port. The node URIs are guaranteed to be constructed before
-     * issuing any requests, so if a UriConstructionException is thrown, then it
-     * is guaranteed that no request was issued.
-     *
-     * @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD)
-     * @param uri the base request URI (up to, but not including, the query
-     * string)
-     * @param parameters the request parameters
-     * @param headers the request headers
-     * @param nodeIdentifiers the NodeIdentifier for each node that the request
-     * should be replaced to
-     *
-     * @return the client response
-     *
-     * @throws NoConnectedNodesException if no nodes are connected as results of
-     * the request
-     * @throws NoResponseFromNodesException if no response could be obtained
-     * @throws UriConstructionException if there was an issue constructing the
-     * URIs tailored for each individual node
-     * @throws ConnectingNodeMutableRequestException if the request was a PUT,
-     * POST, DELETE and a node is connecting to the cluster
-     * @throws DisconnectedNodeMutableRequestException if the request was a PUT,
-     * POST, DELETE and a node is disconnected from the cluster
-     * @throws SafeModeMutableRequestException if the request was a PUT, POST,
-     * DELETE and a the cluster is in safe mode
-     */
-    NodeResponse applyRequest(String method, URI uri, Map<String, List<String>> parameters, Map<String, String> headers,
-            Set<NodeIdentifier> nodeIdentifiers)
-            throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException,
-            DisconnectedNodeMutableRequestException, SafeModeMutableRequestException;
-
-    /**
-     * Federates the HTTP request to all connected nodes in the cluster. The
-     * given URI's host and port will not be used and instead will be adjusted
-     * for each node's host and port. The node URIs are guaranteed to be
-     * constructed before issuing any requests, so if a UriConstructionException
-     * is thrown, then it is guaranteed that no request was issued.
-     *
-     * @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD)
-     * @param uri the base request URI (up to, but not including, the query
-     * string)
-     * @param entity the HTTP request entity
-     * @param headers the request headers
-     *
-     * @return the client response
-     *
-     * @throws NoConnectedNodesException if no nodes are connected as results of
-     * the request
-     * @throws NoResponseFromNodesException if no response could be obtained
-     * @throws UriConstructionException if there was an issue constructing the
-     * URIs tailored for each individual node
-     * @throws ConnectingNodeMutableRequestException if the request was a PUT,
-     * POST, DELETE and a node is connecting to the cluster
-     * @throws DisconnectedNodeMutableRequestException if the request was a PUT,
-     * POST, DELETE and a node is disconnected from the cluster
-     * @throws SafeModeMutableRequestException if the request was a PUT, POST,
-     * DELETE and a the cluster is in safe mode
-     */
-    NodeResponse applyRequest(String method, URI uri, Object entity, Map<String, String> headers)
-            throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException,
-            DisconnectedNodeMutableRequestException, SafeModeMutableRequestException;
-
-    /**
-     * Federates the HTTP request to the nodes specified. The given URI's host
-     * and port will not be used and instead will be adjusted for each node's
-     * host and port. The node URIs are guaranteed to be constructed before
-     * issuing any requests, so if a UriConstructionException is thrown, then it
-     * is guaranteed that no request was issued.
-     *
-     * @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD)
-     * @param uri the base request URI (up to, but not including, the query
-     * string)
-     * @param entity the HTTP request entity
-     * @param headers the request headers
-     * @param nodeIdentifiers the NodeIdentifier for each node that the request
-     * should be replaced to
-     *
-     * @return the client response
-     *
-     * @throws NoConnectedNodesException if no nodes are connected as results of
-     * the request
-     * @throws NoResponseFromNodesException if no response could be obtained
-     * @throws UriConstructionException if there was an issue constructing the
-     * URIs tailored for each individual node
-     * @throws ConnectingNodeMutableRequestException if the request was a PUT,
-     * POST, DELETE and a node is connecting to the cluster
-     * @throws DisconnectedNodeMutableRequestException if the request was a PUT,
-     * POST, DELETE and a node is disconnected from the cluster
-     * @throws SafeModeMutableRequestException if the request was a PUT, POST,
-     * DELETE and a the cluster is in safe mode
-     */
-    NodeResponse applyRequest(String method, URI uri, Object entity, Map<String, String> headers, Set<NodeIdentifier> nodeIdentifiers)
-            throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException,
-            DisconnectedNodeMutableRequestException, SafeModeMutableRequestException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpRequestReplicator.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpRequestReplicator.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpRequestReplicator.java
deleted file mode 100644
index fb57622..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpRequestReplicator.java
+++ /dev/null
@@ -1,99 +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.cluster.manager;
-
-import org.apache.nifi.cluster.manager.exception.UriConstructionException;
-import java.net.URI;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-
-/**
- * A service for managing the replication of requests to nodes. It is up to the
- * implementing class to decide if requests are sent concurrently or serially.
- *
- * Clients must call start() and stop() to initialize and shutdown the instance.
- * The instance must be started before issuing any replication requests.
- *
- * @author unattributed
- */
-public interface HttpRequestReplicator {
-
-    /**
-     * Starts the instance for replicating requests. Start may only be called if
-     * the instance is not running.
-     */
-    void start();
-
-    /**
-     * Stops the instance from replicating requests. Stop may only be called if
-     * the instance is running.
-     */
-    void stop();
-
-    /**
-     * @return true if the instance is started; false otherwise.
-     */
-    boolean isRunning();
-
-    /**
-     * Requests are sent to each node in the cluster. If the request results in
-     * an exception, then the NodeResourceResponse will contain the exception.
-     *
-     * HTTP DELETE and OPTIONS methods must supply an empty parameters map or
-     * else and IllegalArgumentException is thrown.
-     *
-     * @param nodeIds the node identifiers
-     * @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD,
-     * OPTIONS)
-     * @param uri the base request URI (up to, but not including, the query
-     * string)
-     * @param parameters any request parameters
-     * @param headers any HTTP headers
-     *
-     * @return the set of node responses
-     *
-     * @throws UriConstructionException if a request for a node failed to be
-     * constructed from the given prototype URI. If thrown, it is guaranteed
-     * that no request was sent.
-     */
-    Set<NodeResponse> replicate(Set<NodeIdentifier> nodeIds, String method, URI uri, Map<String, List<String>> parameters, Map<String, String> headers) throws UriConstructionException;
-
-    /**
-     * Requests are sent to each node in the cluster. If the request results in
-     * an exception, then the NodeResourceResponse will contain the exception.
-     *
-     * HTTP DELETE, GET, HEAD, and OPTIONS methods will throw an
-     * IllegalArgumentException if used.
-     *
-     * @param nodeIds the node identifiers
-     * @param method the HTTP method (e.g., POST, PUT)
-     * @param uri the base request URI (up to, but not including, the query
-     * string)
-     * @param entity an entity
-     * @param headers any HTTP headers
-     *
-     * @return the set of node responses
-     *
-     * @throws UriConstructionException if a request for a node failed to be
-     * constructed from the given prototype URI. If thrown, it is guaranteed
-     * that no request was sent.
-     */
-    Set<NodeResponse> replicate(Set<NodeIdentifier> nodeIds, String method, URI uri, Object entity, Map<String, String> headers) throws UriConstructionException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpResponseMapper.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpResponseMapper.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpResponseMapper.java
deleted file mode 100644
index 843a666..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpResponseMapper.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.cluster.manager;
-
-import java.net.URI;
-import java.util.Map;
-import java.util.Set;
-import org.apache.nifi.cluster.node.Node.Status;
-
-/**
- * Maps a HTTP response to a node status.
- *
- * @author unattributed
- */
-public interface HttpResponseMapper {
-
-    /**
-     * Maps a HTTP response to a node response and the corresponding node
-     * status.
-     *
-     * @param requestURI the original request URI
-     * @param nodeResponses a set of node resource responses
-     *
-     * @return a map associating the node response to the node status
-     */
-    Map<NodeResponse, Status> map(URI requestURI, Set<NodeResponse> nodeResponses);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java
deleted file mode 100644
index 3f966e5..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java
+++ /dev/null
@@ -1,329 +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.cluster.manager;
-
-import com.sun.jersey.api.client.ClientResponse;
-import java.io.BufferedInputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.net.URI;
-import java.util.List;
-import java.util.Objects;
-import java.util.concurrent.TimeUnit;
-
-import javax.ws.rs.HttpMethod;
-import javax.ws.rs.WebApplicationException;
-import javax.ws.rs.core.Response;
-import javax.ws.rs.core.Response.ResponseBuilder;
-import javax.ws.rs.core.Response.Status;
-import javax.ws.rs.core.StreamingOutput;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.web.api.entity.Entity;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Encapsulates a node's response in regards to receiving a external API
- * request.
- *
- * Both the ClientResponse and (server) Response may be obtained from this
- * instance. The ClientResponse is stored as it is received from the node. This
- * includes the entity input stream. The Response is constructed on demand when
- * mapping a ClientResponse to the Response. The ClientResponse to Response
- * mapping includes copying the ClientResponse's input stream to the Response.
- * Therefore, the getResponse() method should not be called more than once.
- * Furthermore, the method should not be called if the caller has already read
- * the ClientResponse's input stream.
- *
- * If a ClientResponse was unable to be created, then a NodeResponse will store
- * the Throwable, which may be obtained by calling getThrowable().
- *
- * This class overrides hashCode and equals and considers two instances to be
- * equal if they have the equal NodeIdentifiers.
- *
- * @author unattributed
- */
-public class NodeResponse {
-
-    private static final Logger logger = LoggerFactory.getLogger(NodeResponse.class);
-    private final String httpMethod;
-    private final URI requestUri;
-    private final ClientResponse clientResponse;
-    private final NodeIdentifier nodeId;
-    private final Throwable throwable;
-    private boolean hasCreatedResponse = false;
-    private final Entity updatedEntity;
-    private final long requestDurationNanos;
-    private final String requestId;
-
-    public NodeResponse(final NodeIdentifier nodeId, final String httpMethod, final URI requestUri, final ClientResponse clientResponse, final long requestDurationNanos, final String requestId) {
-        if (nodeId == null) {
-            throw new IllegalArgumentException("Node identifier may not be null.");
-        } else if (StringUtils.isBlank(httpMethod)) {
-            throw new IllegalArgumentException("Http method may not be null or empty.");
-        } else if (requestUri == null) {
-            throw new IllegalArgumentException("Request URI may not be null.");
-        } else if (clientResponse == null) {
-            throw new IllegalArgumentException("ClientResponse may not be null.");
-        }
-        this.nodeId = nodeId;
-        this.httpMethod = httpMethod;
-        this.requestUri = requestUri;
-        this.clientResponse = clientResponse;
-        this.throwable = null;
-        this.updatedEntity = null;
-        this.requestDurationNanos = requestDurationNanos;
-        this.requestId = requestId;
-    }
-
-    public NodeResponse(final NodeIdentifier nodeId, final String httpMethod, final URI requestUri, final Throwable throwable) {
-        if (nodeId == null) {
-            throw new IllegalArgumentException("Node identifier may not be null.");
-        } else if (StringUtils.isBlank(httpMethod)) {
-            throw new IllegalArgumentException("Http method may not be null or empty.");
-        } else if (requestUri == null) {
-            throw new IllegalArgumentException("Request URI may not be null.");
-        } else if (throwable == null) {
-            throw new IllegalArgumentException("Throwable may not be null.");
-        }
-        this.nodeId = nodeId;
-        this.httpMethod = httpMethod;
-        this.requestUri = requestUri;
-        this.clientResponse = null;
-        this.throwable = throwable;
-        this.updatedEntity = null;
-        this.requestDurationNanos = -1L;
-        this.requestId = null;
-    }
-
-    public NodeResponse(final NodeResponse example, final Entity updatedEntity) {
-        Objects.requireNonNull(example, "NodeResponse cannot be null");
-        Objects.requireNonNull(updatedEntity, "UpdatedEntity cannot be null");
-
-        this.nodeId = example.nodeId;
-        this.httpMethod = example.httpMethod;
-        this.requestUri = example.requestUri;
-        this.clientResponse = example.clientResponse;
-        this.throwable = example.throwable;
-        this.updatedEntity = updatedEntity;
-        this.requestDurationNanos = example.requestDurationNanos;
-        this.requestId = null;
-    }
-
-    public NodeIdentifier getNodeId() {
-        return nodeId;
-    }
-
-    public String getHttpMethod() {
-        return httpMethod;
-    }
-
-    public URI getRequestUri() {
-        return requestUri;
-    }
-
-    /**
-     * @return the HTTP response status code
-     */
-    public int getStatus() {
-        if (hasThrowable()) {
-            /*
-             * since there is a throwable, there is no client input stream to 
-             * worry about maintaining, so we can call getResponse() method
-             */
-            return getResponse().getStatus();
-        } else {
-            /*
-             * use client response's status instead of calling getResponse().getStatus()
-             * so that we don't read the client's input stream as part of creating 
-             * the response in the getResponse() method
-             */
-            return clientResponse.getStatus();
-        }
-    }
-
-    /**
-     * Returns true if the response status is 2xx, false otherwise.
-     *
-     * @return
-     */
-    public boolean is2xx() {
-        final int statusCode = getStatus();
-        return (200 <= statusCode && statusCode <= 299);
-    }
-
-    /**
-     * Returns true if the response status is 5xx, false otherwise.
-     *
-     * @return
-     */
-    public boolean is5xx() {
-        final int statusCode = getStatus();
-        return (500 <= statusCode && statusCode <= 599);
-    }
-
-    /**
-     * Returns null if hasThrowable() is true; otherwise the client's response
-     * is returned.
-     *
-     * The ClientResponse's input stream can only be read once.
-     *
-     * @return the client's response
-     */
-    public ClientResponse getClientResponse() {
-        return clientResponse;
-    }
-
-    /**
-     * Creates a Response by mapping the ClientResponse values to it. Since the
-     * ClientResponse's input stream can only be read once, this method should
-     * only be called once. Furthermore, the caller should not have already read
-     * the ClientResponse's input stream.
-     *
-     * @return the response
-     */
-    public Response getResponse() {
-        // if the response encapsulates a throwable, then the input stream is never read and the below warning is irrelevant
-        if (hasCreatedResponse && !hasThrowable()) {
-            logger.warn("ClientResponse's input stream has already been read.  The created response will not contain this data.");
-        }
-        hasCreatedResponse = true;
-        return createResponse();
-    }
-
-    /**
-     * Returns the throwable or null if no throwable exists.
-     *
-     * @return the throwable or null if no throwable exists
-     */
-    public Throwable getThrowable() {
-        return throwable;
-    }
-
-    /**
-     * Returns true if a throwable was thrown and a response was not able to be
-     * created; false otherwise.
-     *
-     * @return true if a throwable was thrown and a response was not able to be
-     * created; false otherwise
-     */
-    public boolean hasThrowable() {
-        return getThrowable() != null;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (getClass() != obj.getClass()) {
-            return false;
-        }
-        final NodeResponse other = (NodeResponse) obj;
-        if (this.nodeId != other.nodeId && (this.nodeId == null || !this.nodeId.equals(other.nodeId))) {
-            return false;
-        }
-        return true;
-    }
-
-    @Override
-    public int hashCode() {
-        int hash = 7;
-        hash = 13 * hash + (this.nodeId != null ? this.nodeId.hashCode() : 0);
-        return hash;
-    }
-
-    public long getRequestDuration(final TimeUnit timeUnit) {
-        return timeUnit.convert(requestDurationNanos, TimeUnit.NANOSECONDS);
-    }
-
-    public String getRequestId() {
-        return requestId;
-    }
-
-    private Response createResponse() {
-
-        // if no client response was created, then generate a 500 response
-        if (hasThrowable()) {
-            return Response.status(Status.INTERNAL_SERVER_ERROR).build();
-        }
-
-        // set the status
-        final ResponseBuilder responseBuilder = Response.status(clientResponse.getStatus());
-
-        // set the headers
-        for (final String key : clientResponse.getHeaders().keySet()) {
-            final List<String> values = clientResponse.getHeaders().get(key);
-            for (final String value : values) {
-
-                if (key.equalsIgnoreCase("transfer-encoding") || key.equalsIgnoreCase("content-length")) {
-                    /*
-                     * do not copy the transfer-encoding header (i.e., chunked encoding) or
-                     * the content-length. Let the outgoing response builder determine it.
-                     */
-                    continue;
-                } else if (key.equals("X-ClusterContext")) {
-                    /*
-                     * do not copy the cluster context to the response because
-                     * this information is private and should not be sent to
-                     * the client
-                     */
-                    continue;
-                }
-                responseBuilder.header(key, value);
-            }
-        }
-
-        // head requests must not have a message-body in the response
-        if (!HttpMethod.HEAD.equalsIgnoreCase(httpMethod)) {
-
-            // set the entity
-            if (updatedEntity == null) {
-                responseBuilder.entity(new StreamingOutput() {
-                    @Override
-                    public void write(final OutputStream output) throws IOException, WebApplicationException {
-                        BufferedInputStream bis = null;
-                        try {
-                            bis = new BufferedInputStream(clientResponse.getEntityInputStream());
-                            IOUtils.copy(bis, output);
-                        } finally {
-                            IOUtils.closeQuietly(bis);
-                        }
-                    }
-                });
-            } else {
-                responseBuilder.entity(updatedEntity);
-            }
-        }
-
-        return responseBuilder.build();
-    }
-
-    @Override
-    public String toString() {
-        final StringBuilder sb = new StringBuilder();
-        sb.append("NodeResponse[nodeUri=").append(nodeId.getApiAddress()).append(":").append(nodeId.getApiPort()).append(",")
-                .append("method=").append(httpMethod)
-                .append(",URI=").append(requestUri)
-                .append(",ResponseCode=").append(getStatus())
-                .append(",Duration=").append(TimeUnit.MILLISECONDS.convert(requestDurationNanos, TimeUnit.NANOSECONDS)).append(" ms]");
-        return sb.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/BlockedByFirewallException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/BlockedByFirewallException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/BlockedByFirewallException.java
deleted file mode 100644
index 49bcd35..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/BlockedByFirewallException.java
+++ /dev/null
@@ -1,60 +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.cluster.manager.exception;
-
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-
-/**
- *
- */
-public class BlockedByFirewallException extends ClusterException {
-
-    private final NodeIdentifier nodeId;
-    private final boolean isExistingNode;
-
-    public BlockedByFirewallException(NodeIdentifier nodeId, boolean isExistingNode, String msg, Throwable cause) {
-        super(msg, cause);
-        this.nodeId = nodeId;
-        this.isExistingNode = isExistingNode;
-    }
-
-    public BlockedByFirewallException(NodeIdentifier nodeId, boolean isExistingNode, Throwable cause) {
-        super(cause);
-        this.nodeId = nodeId;
-        this.isExistingNode = isExistingNode;
-    }
-
-    public BlockedByFirewallException(NodeIdentifier nodeId, boolean isExistingNode, String msg) {
-        super(msg);
-        this.nodeId = nodeId;
-        this.isExistingNode = isExistingNode;
-    }
-
-    public BlockedByFirewallException(NodeIdentifier nodeId, boolean isExistingNode) {
-        this.nodeId = nodeId;
-        this.isExistingNode = isExistingNode;
-    }
-
-    public NodeIdentifier getNodeId() {
-        return nodeId;
-    }
-
-    public boolean isExistingNode() {
-        return isExistingNode;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ClusterException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ClusterException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ClusterException.java
deleted file mode 100644
index 3bf9752..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ClusterException.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.cluster.manager.exception;
-
-/**
- * The base exception class for cluster related exceptions.
- *
- * @author unattributed
- */
-public class ClusterException extends RuntimeException {
-
-    public ClusterException() {
-    }
-
-    public ClusterException(String msg) {
-        super(msg);
-    }
-
-    public ClusterException(Throwable cause) {
-        super(cause);
-    }
-
-    public ClusterException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}


[49/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java
----------------------------------------------------------------------
diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java
deleted file mode 100644
index cfbae88..0000000
--- a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java
+++ /dev/null
@@ -1,752 +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.provenance;
-
-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;
-import org.apache.nifi.processor.Relationship;
-
-/**
- * Holder for provenance relevant information
- * <p/>
- * @author none
- */
-public final class StandardProvenanceEventRecord implements ProvenanceEventRecord {
-
-    private final long eventTime;
-    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;
-    private final String sourceSystemFlowFileIdentifier;
-    private final String uuid;
-    private final List<String> parentUuids;
-    private final List<String> childrenUuids;
-    private final String alternateIdentifierUri;
-    private final String details;
-    private final String relationship;
-    private final long storageByteOffset;
-    private final String storageFilename;
-    private final long eventDuration;
-
-    private final String contentClaimSection;
-    private final String contentClaimContainer;
-    private final String contentClaimIdentifier;
-    private final Long contentClaimOffset;
-    private final long contentSize;
-
-    private final String previousClaimSection;
-    private final String previousClaimContainer;
-    private final String previousClaimIdentifier;
-    private final Long previousClaimOffset;
-    private final Long previousSize;
-
-    private final String sourceQueueIdentifier;
-
-    private final Map<String, String> previousAttributes;
-    private final Map<String, String> updatedAttributes;
-
-    private volatile long eventId;
-
-    private StandardProvenanceEventRecord(final Builder builder) {
-        this.eventTime = builder.eventTime;
-        this.entryDate = builder.entryDate;
-        this.eventType = builder.eventType;
-        this.componentId = builder.componentId;
-        this.componentType = builder.componentType;
-        this.transitUri = builder.transitUri;
-        this.sourceSystemFlowFileIdentifier = builder.sourceSystemFlowFileIdentifier;
-        this.uuid = builder.uuid;
-        this.parentUuids = builder.parentUuids;
-        this.childrenUuids = builder.childrenUuids;
-        this.alternateIdentifierUri = builder.alternateIdentifierUri;
-        this.details = builder.details;
-        this.relationship = builder.relationship;
-        this.storageByteOffset = builder.storageByteOffset;
-        this.storageFilename = builder.storageFilename;
-        this.eventDuration = builder.eventDuration;
-        this.lineageStartDate = builder.lineageStartDate;
-        this.lineageIdentifiers = Collections.unmodifiableSet(builder.lineageIdentifiers);
-
-        previousClaimSection = builder.previousClaimSection;
-        previousClaimContainer = builder.previousClaimContainer;
-        previousClaimIdentifier = builder.previousClaimIdentifier;
-        previousClaimOffset = builder.previousClaimOffset;
-        previousSize = builder.previousSize;
-
-        contentClaimSection = builder.contentClaimSection;
-        contentClaimContainer = builder.contentClaimContainer;
-        contentClaimIdentifier = builder.contentClaimIdentifier;
-        contentClaimOffset = builder.contentClaimOffset;
-        contentSize = builder.contentSize;
-
-        previousAttributes = builder.previousAttributes == null ? Collections.<String, String>emptyMap() : Collections.unmodifiableMap(builder.previousAttributes);
-        updatedAttributes = builder.updatedAttributes == null ? Collections.<String, String>emptyMap() : Collections.unmodifiableMap(builder.updatedAttributes);
-
-        sourceQueueIdentifier = builder.sourceQueueIdentifier;
-
-    }
-
-    public String getStorageFilename() {
-        return storageFilename;
-    }
-
-    public long getStorageByteOffset() {
-        return storageByteOffset;
-    }
-
-    void setEventId(final long eventId) {
-        this.eventId = eventId;
-    }
-
-    @Override
-    public long getEventId() {
-        return eventId;
-    }
-
-    @Override
-    public long getEventTime() {
-        return eventTime;
-    }
-
-    @Override
-    public Set<String> getLineageIdentifiers() {
-        return lineageIdentifiers;
-    }
-
-    @Override
-    public long getLineageStartDate() {
-        return lineageStartDate;
-    }
-
-    @Override
-    public long getFileSize() {
-        return contentSize;
-    }
-
-    @Override
-    public Long getPreviousFileSize() {
-        return previousSize;
-    }
-
-    @Override
-    public ProvenanceEventType getEventType() {
-        return eventType;
-    }
-
-    @Override
-    public Map<String, String> getAttributes() {
-        final Map<String, String> allAttrs = new HashMap<>(previousAttributes.size() + updatedAttributes.size());
-        allAttrs.putAll(previousAttributes);
-        for (final Map.Entry<String, String> entry : updatedAttributes.entrySet()) {
-            if (entry.getValue() != null) {
-                allAttrs.put(entry.getKey(), entry.getValue());
-            }
-        }
-        return allAttrs;
-    }
-
-    @Override
-    public String getComponentId() {
-        return componentId;
-    }
-
-    @Override
-    public String getComponentType() {
-        return componentType;
-    }
-
-    @Override
-    public String getTransitUri() {
-        return transitUri;
-    }
-
-    @Override
-    public String getSourceSystemFlowFileIdentifier() {
-        return sourceSystemFlowFileIdentifier;
-    }
-
-    @Override
-    public String getFlowFileUuid() {
-        return uuid;
-    }
-
-    @Override
-    public List<String> getParentUuids() {
-        return parentUuids == null ? Collections.<String>emptyList() : parentUuids;
-    }
-
-    @Override
-    public List<String> getChildUuids() {
-        return childrenUuids == null ? Collections.<String>emptyList() : childrenUuids;
-    }
-
-    @Override
-    public String getAlternateIdentifierUri() {
-        return alternateIdentifierUri;
-    }
-
-    @Override
-    public long getEventDuration() {
-        return eventDuration;
-    }
-
-    @Override
-    public String getDetails() {
-        return details;
-    }
-
-    @Override
-    public String getRelationship() {
-        return relationship;
-    }
-
-    @Override
-    public long getFlowFileEntryDate() {
-        return entryDate;
-    }
-
-    @Override
-    public String getContentClaimSection() {
-        return contentClaimSection;
-    }
-
-    @Override
-    public String getContentClaimContainer() {
-        return contentClaimContainer;
-    }
-
-    @Override
-    public String getContentClaimIdentifier() {
-        return contentClaimIdentifier;
-    }
-
-    @Override
-    public Long getContentClaimOffset() {
-        return contentClaimOffset;
-    }
-
-    @Override
-    public String getSourceQueueIdentifier() {
-        return sourceQueueIdentifier;
-    }
-
-    @Override
-    public Map<String, String> getPreviousAttributes() {
-        return previousAttributes;
-    }
-
-    @Override
-    public String getPreviousContentClaimContainer() {
-        return previousClaimContainer;
-    }
-
-    @Override
-    public String getPreviousContentClaimIdentifier() {
-        return previousClaimIdentifier;
-    }
-
-    @Override
-    public Long getPreviousContentClaimOffset() {
-        return previousClaimOffset;
-    }
-
-    @Override
-    public String getPreviousContentClaimSection() {
-        return previousClaimSection;
-    }
-
-    @Override
-    public Map<String, String> getUpdatedAttributes() {
-        return updatedAttributes;
-    }
-
-    @Override
-    public int hashCode() {
-        final int eventTypeCode;
-        if (eventType == ProvenanceEventType.CLONE || eventType == ProvenanceEventType.JOIN || eventType == ProvenanceEventType.FORK) {
-            eventTypeCode = 1472;
-        } else if (eventType == ProvenanceEventType.REPLAY) {
-            eventTypeCode = 21479 + (int) (0x7FFFFFFF & eventTime); // use lower bits of event time.
-        } else {
-            eventTypeCode = 4812 + eventType.hashCode() + 4 * uuid.hashCode();
-        }
-
-        return -37423 + 3 * componentId.hashCode() + (transitUri == null ? 0 : 41 * transitUri.hashCode())
-                + (relationship == null ? 0 : 47 * relationship.hashCode()) + 44 * eventTypeCode;
-    }
-
-    @Override
-    public boolean equals(final Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (obj == this) {
-            return true;
-        }
-        if (!(obj instanceof StandardProvenanceEventRecord)) {
-            return false;
-        }
-
-        final StandardProvenanceEventRecord other = (StandardProvenanceEventRecord) obj;
-        // If event ID's are populated and not equal, return false. If they have not yet been populated, do not
-        // use them in the comparison.
-        if (eventId > 0L && other.getEventId() > 0L && eventId != other.getEventId()) {
-            return false;
-        }
-        if (eventType != other.eventType) {
-            return false;
-        }
-
-        if (!componentId.equals(other.componentId)) {
-            return false;
-        }
-
-        if (different(parentUuids, other.parentUuids)) {
-            return false;
-        }
-
-        if (different(childrenUuids, other.childrenUuids)) {
-            return false;
-        }
-
-        // SPAWN had issues indicating which should be the event's FlowFileUUID in the case that there is 1 parent and 1 child.
-        if (!uuid.equals(other.uuid)) {
-            return false;
-        }
-
-        if (different(transitUri, other.transitUri)) {
-            return false;
-        }
-
-        if (different(relationship, other.relationship)) {
-            return false;
-        }
-
-        return !(eventType == ProvenanceEventType.REPLAY && eventTime != other.getEventTime());
-    }
-
-    private boolean different(final Object a, final Object b) {
-        if (a == null && b == null) {
-            return false;
-        }
-        if (a == null || b == null) {
-            return true;
-        }
-
-        return !a.equals(b);
-    }
-
-    private boolean different(final List<String> a, final List<String> b) {
-        if (a == null && b == null) {
-            return false;
-        }
-
-        if (a == null && b != null) {
-            return true;
-        }
-
-        if (a != null && b == null) {
-            return true;
-        }
-
-        if (a.size() != b.size()) {
-            return true;
-        }
-
-        final List<String> sortedA = new ArrayList<>(a);
-        final List<String> sortedB = new ArrayList<>(b);
-
-        Collections.sort(sortedA);
-        Collections.sort(sortedB);
-
-        for (int i = 0; i < sortedA.size(); i++) {
-            if (!sortedA.get(i).equals(sortedB.get(i))) {
-                return true;
-            }
-        }
-
-        return false;
-    }
-
-    @Override
-    public String toString() {
-        return "ProvenanceEventRecord ["
-                + "eventId=" + eventId
-                + ", eventType=" + eventType
-                + ", eventTime=" + new Date(eventTime)
-                + ", uuid=" + uuid
-                + ", fileSize=" + contentSize
-                + ", componentId=" + componentId
-                + ", transitUri=" + transitUri
-                + ", sourceSystemFlowFileIdentifier=" + sourceSystemFlowFileIdentifier
-                + ", parentUuids=" + parentUuids
-                + ", alternateIdentifierUri=" + alternateIdentifierUri + "]";
-    }
-
-    public static class Builder implements ProvenanceEventBuilder {
-
-        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;
-        private String sourceSystemFlowFileIdentifier = null;
-        private String transitUri = null;
-        private String uuid = null;
-        private List<String> parentUuids = null;
-        private List<String> childrenUuids = null;
-        private String contentType = null;
-        private String alternateIdentifierUri = null;
-        private String details = null;
-        private String relationship = null;
-        private long storageByteOffset = -1L;
-        private long eventDuration = -1L;
-        private String storageFilename;
-
-        private String contentClaimSection;
-        private String contentClaimContainer;
-        private String contentClaimIdentifier;
-        private Long contentClaimOffset;
-        private Long contentSize;
-
-        private String previousClaimSection;
-        private String previousClaimContainer;
-        private String previousClaimIdentifier;
-        private Long previousClaimOffset;
-        private Long previousSize;
-
-        private String sourceQueueIdentifier;
-
-        private Map<String, String> previousAttributes;
-        private Map<String, String> updatedAttributes;
-
-        @Override
-        public Builder fromEvent(final ProvenanceEventRecord event) {
-            eventTime = event.getEventTime();
-            entryDate = event.getFlowFileEntryDate();
-            lineageStartDate = event.getLineageStartDate();
-            lineageIdentifiers = event.getLineageIdentifiers();
-            eventType = event.getEventType();
-            componentId = event.getComponentId();
-            componentType = event.getComponentType();
-            transitUri = event.getTransitUri();
-            sourceSystemFlowFileIdentifier = event.getSourceSystemFlowFileIdentifier();
-            uuid = event.getFlowFileUuid();
-            parentUuids = event.getParentUuids();
-            childrenUuids = event.getChildUuids();
-            alternateIdentifierUri = event.getAlternateIdentifierUri();
-            eventDuration = event.getEventDuration();
-            previousAttributes = event.getPreviousAttributes();
-            updatedAttributes = event.getUpdatedAttributes();
-            details = event.getDetails();
-            relationship = event.getRelationship();
-
-            contentClaimSection = event.getContentClaimSection();
-            contentClaimContainer = event.getContentClaimContainer();
-            contentClaimIdentifier = event.getContentClaimIdentifier();
-            contentClaimOffset = event.getContentClaimOffset();
-            contentSize = event.getFileSize();
-
-            previousClaimSection = event.getPreviousContentClaimSection();
-            previousClaimContainer = event.getPreviousContentClaimContainer();
-            previousClaimIdentifier = event.getPreviousContentClaimIdentifier();
-            previousClaimOffset = event.getPreviousContentClaimOffset();
-            previousSize = event.getPreviousFileSize();
-
-            sourceQueueIdentifier = event.getSourceQueueIdentifier();
-
-            if (event instanceof StandardProvenanceEventRecord) {
-                final StandardProvenanceEventRecord standardProvEvent = (StandardProvenanceEventRecord) event;
-                storageByteOffset = standardProvEvent.storageByteOffset;
-                storageFilename = standardProvEvent.storageFilename;
-            }
-
-            return this;
-        }
-
-        @Override
-        public Builder setFlowFileEntryDate(final long entryDate) {
-            this.entryDate = entryDate;
-            return this;
-        }
-
-        @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;
-            return this;
-        }
-
-        @Override
-        public Builder setFlowFileUUID(final String uuid) {
-            this.uuid = uuid;
-            return this;
-        }
-
-        public Builder setStorageLocation(final String filename, final long offset) {
-            this.storageFilename = filename;
-            this.storageByteOffset = offset;
-            return this;
-        }
-
-        @Override
-        public Builder setEventTime(long eventTime) {
-            this.eventTime = eventTime;
-            return this;
-        }
-
-        @Override
-        public Builder setEventDuration(final long millis) {
-            this.eventDuration = millis;
-            return this;
-        }
-
-        @Override
-        public Builder setLineageStartDate(final long startDate) {
-            this.lineageStartDate = startDate;
-            return this;
-        }
-
-        public Builder addLineageIdentifier(final String lineageIdentifier) {
-            this.lineageIdentifiers.add(lineageIdentifier);
-            return this;
-        }
-
-        @Override
-        public Builder setEventType(ProvenanceEventType eventType) {
-            this.eventType = eventType;
-            return this;
-        }
-
-        @Override
-        public Builder setComponentId(String componentId) {
-            this.componentId = componentId;
-            return this;
-        }
-
-        @Override
-        public Builder setComponentType(String componentType) {
-            this.componentType = componentType;
-            return this;
-        }
-
-        @Override
-        public Builder setSourceSystemFlowFileIdentifier(String sourceSystemFlowFileIdentifier) {
-            this.sourceSystemFlowFileIdentifier = sourceSystemFlowFileIdentifier;
-            return this;
-        }
-
-        @Override
-        public Builder setTransitUri(String transitUri) {
-            this.transitUri = transitUri;
-            return this;
-        }
-
-        @Override
-        public Builder addParentFlowFile(final FlowFile parentFlowFile) {
-            if (this.parentUuids == null) {
-                this.parentUuids = new ArrayList<>();
-            }
-            this.parentUuids.add(parentFlowFile.getAttribute(CoreAttributes.UUID.key()));
-            return this;
-        }
-
-        public Builder addParentUuid(final String uuid) {
-            if (this.parentUuids == null) {
-                this.parentUuids = new ArrayList<>();
-            }
-            this.parentUuids.add(uuid);
-            return this;
-        }
-
-        @Override
-        public Builder removeParentFlowFile(final FlowFile parentFlowFile) {
-            if (this.parentUuids == null) {
-                return this;
-            }
-
-            parentUuids.remove(parentFlowFile.getAttribute(CoreAttributes.UUID.key()));
-            return this;
-        }
-
-        @Override
-        public Builder addChildFlowFile(final FlowFile childFlowFile) {
-            if (this.childrenUuids == null) {
-                this.childrenUuids = new ArrayList<>();
-            }
-            this.childrenUuids.add(childFlowFile.getAttribute(CoreAttributes.UUID.key()));
-            return this;
-        }
-
-        public Builder addChildUuid(final String uuid) {
-            if (this.childrenUuids == null) {
-                this.childrenUuids = new ArrayList<>();
-            }
-            this.childrenUuids.add(uuid);
-            return this;
-        }
-
-        @Override
-        public Builder removeChildFlowFile(final FlowFile childFlowFile) {
-            if (this.childrenUuids == null) {
-                return this;
-            }
-
-            childrenUuids.remove(childFlowFile.getAttribute(CoreAttributes.UUID.key()));
-            return this;
-        }
-
-        public Builder setContentType(String contentType) {
-            this.contentType = contentType;
-            return this;
-        }
-
-        @Override
-        public Builder setAlternateIdentifierUri(String alternateIdentifierUri) {
-            this.alternateIdentifierUri = alternateIdentifierUri;
-            return this;
-        }
-
-        @Override
-        public Builder setDetails(String details) {
-            this.details = details;
-            return this;
-        }
-
-        @Override
-        public Builder setRelationship(Relationship relationship) {
-            this.relationship = relationship.getName();
-            return this;
-        }
-
-        public Builder setRelationship(final String relationship) {
-            this.relationship = relationship;
-            return this;
-        }
-
-        @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());
-            this.contentSize = flowFile.getSize();
-            return this;
-        }
-
-        @Override
-        public Builder setPreviousContentClaim(final String container, final String section, final String identifier, final Long offset, final long size) {
-            previousClaimSection = section;
-            previousClaimContainer = container;
-            previousClaimIdentifier = identifier;
-            previousClaimOffset = offset;
-            previousSize = size;
-            return this;
-        }
-
-        @Override
-        public Builder setCurrentContentClaim(final String container, final String section, final String identifier, final Long offset, final long size) {
-            contentClaimSection = section;
-            contentClaimContainer = container;
-            contentClaimIdentifier = identifier;
-            contentClaimOffset = offset;
-            contentSize = size;
-            return this;
-        }
-
-        @Override
-        public Builder setSourceQueueIdentifier(final String identifier) {
-            sourceQueueIdentifier = identifier;
-            return this;
-        }
-
-        private void assertSet(final Object value, final String name) {
-            if (value == null) {
-                throw new IllegalStateException("Cannot create Provenance Event Record because " + name + " is not set");
-            }
-        }
-
-        public ProvenanceEventType getEventType() {
-            return eventType;
-        }
-
-        public List<String> getChildUuids() {
-            return Collections.unmodifiableList(childrenUuids);
-        }
-
-        public List<String> getParentUuids() {
-            return Collections.unmodifiableList(parentUuids);
-        }
-
-        @Override
-        public StandardProvenanceEventRecord build() {
-            assertSet(eventType, "Event Type");
-            assertSet(componentId, "Component ID");
-            assertSet(componentType, "Component Type");
-            assertSet(uuid, "FlowFile UUID");
-            assertSet(contentSize, "FlowFile Size");
-
-            switch (eventType) {
-                case ADDINFO:
-                    if (alternateIdentifierUri == null) {
-                        throw new IllegalStateException("Cannot create Provenance Event Record of type " + eventType + " because no alternate identifiers have been set");
-                    }
-                    break;
-                case RECEIVE:
-                case SEND:
-                    assertSet(transitUri, "Transit URI");
-                    break;
-                case ROUTE:
-                    assertSet(relationship, "Relationship");
-                    break;
-                case CLONE:
-                case FORK:
-                case JOIN:
-                    if ((parentUuids == null || parentUuids.isEmpty()) && (childrenUuids == null || childrenUuids.isEmpty())) {
-                        throw new IllegalStateException("Cannot create Provenance Event Record of type " + eventType + " because no Parent UUIDs or Children UUIDs have been set");
-                    }
-                    break;
-                default:
-                    break;
-            }
-
-            return new StandardProvenanceEventRecord(this);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java
----------------------------------------------------------------------
diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java
deleted file mode 100644
index 9a9a27d..0000000
--- a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java
+++ /dev/null
@@ -1,168 +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.provenance;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import org.apache.nifi.provenance.search.Query;
-import org.apache.nifi.provenance.search.QueryResult;
-
-public class StandardQueryResult implements QueryResult {
-
-    public static final int TTL = (int) TimeUnit.MILLISECONDS.convert(30, TimeUnit.MINUTES);
-    private final Query query;
-    private final long creationNanos;
-
-    private final int numSteps;
-    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
-    private final Lock readLock = rwLock.readLock();
-
-    private final Lock writeLock = rwLock.writeLock();
-    // guarded by writeLock
-    private final List<ProvenanceEventRecord> matchingRecords = new ArrayList<>();
-    private long totalHitCount;
-    private int numCompletedSteps = 0;
-    private Date expirationDate;
-    private String error;
-    private long queryTime;
-
-    private volatile boolean canceled = false;
-
-    public StandardQueryResult(final Query query, final int numSteps) {
-        this.query = query;
-        this.numSteps = numSteps;
-        this.creationNanos = System.nanoTime();
-
-        updateExpiration();
-    }
-
-    @Override
-    public List<ProvenanceEventRecord> getMatchingEvents() {
-        readLock.lock();
-        try {
-            if (matchingRecords.size() <= query.getMaxResults()) {
-                return new ArrayList<>(matchingRecords);
-            }
-
-            final List<ProvenanceEventRecord> copy = new ArrayList<>(query.getMaxResults());
-            for (int i = 0; i < query.getMaxResults(); i++) {
-                copy.add(matchingRecords.get(i));
-            }
-
-            return copy;
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public long getTotalHitCount() {
-        readLock.lock();
-        try {
-            return totalHitCount;
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public long getQueryTime() {
-        return queryTime;
-    }
-
-    @Override
-    public Date getExpiration() {
-        return expirationDate;
-    }
-
-    @Override
-    public String getError() {
-        return error;
-    }
-
-    @Override
-    public int getPercentComplete() {
-        readLock.lock();
-        try {
-            return (numSteps < 1) ? 100 : (int) (((float) numCompletedSteps / (float) numSteps) * 100.0F);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public boolean isFinished() {
-        readLock.lock();
-        try {
-            return numCompletedSteps >= numSteps || canceled;
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    void cancel() {
-        this.canceled = true;
-    }
-
-    public void setError(final String error) {
-        writeLock.lock();
-        try {
-            this.error = error;
-            numCompletedSteps++;
-
-            updateExpiration();
-            if (numCompletedSteps >= numSteps) {
-                final long searchNanos = System.nanoTime() - creationNanos;
-                queryTime = TimeUnit.MILLISECONDS.convert(searchNanos, TimeUnit.NANOSECONDS);
-            }
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    public void update(final Collection<ProvenanceEventRecord> matchingRecords, final long totalHits) {
-        writeLock.lock();
-        try {
-            this.matchingRecords.addAll(matchingRecords);
-            this.totalHitCount += totalHits;
-
-            numCompletedSteps++;
-            updateExpiration();
-
-            if (numCompletedSteps >= numSteps) {
-                final long searchNanos = System.nanoTime() - creationNanos;
-                queryTime = TimeUnit.MILLISECONDS.convert(searchNanos, TimeUnit.NANOSECONDS);
-            }
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * Must be called with write lock!
-     */
-    private void updateExpiration() {
-        expirationDate = new Date(System.currentTimeMillis() + TTL);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EdgeNode.java
----------------------------------------------------------------------
diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EdgeNode.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EdgeNode.java
deleted file mode 100644
index 0aaf5ef..0000000
--- a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EdgeNode.java
+++ /dev/null
@@ -1,74 +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.provenance.lineage;
-
-import static java.util.Objects.requireNonNull;
-
-public class EdgeNode implements LineageEdge {
-
-    private final String uuid;
-    private final LineageNode source;
-    private final LineageNode destination;
-
-    public EdgeNode(final String uuid, final LineageNode source, final LineageNode destination) {
-        this.uuid = uuid;
-        this.source = requireNonNull(source);
-        this.destination = requireNonNull(destination);
-    }
-
-    @Override
-    public String getUuid() {
-        return uuid;
-    }
-
-    @Override
-    public LineageNode getSource() {
-        return source;
-    }
-
-    @Override
-    public LineageNode getDestination() {
-        return destination;
-    }
-
-    @Override
-    public int hashCode() {
-        return 43298293 + source.hashCode() + destination.hashCode();
-    }
-
-    @Override
-    public boolean equals(final Object obj) {
-        if (this == obj) {
-            return true;
-        }
-        if (obj == null) {
-            return false;
-        }
-
-        if (!(obj instanceof EdgeNode)) {
-            return false;
-        }
-
-        final EdgeNode other = (EdgeNode) obj;
-        return (source.equals(other.source) && destination.equals(other.destination));
-    }
-
-    @Override
-    public String toString() {
-        return "Edge[Source=" + source + ", Destination=" + destination + "]";
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java
----------------------------------------------------------------------
diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java
deleted file mode 100644
index 12d9a4f..0000000
--- a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java
+++ /dev/null
@@ -1,109 +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.provenance.lineage;
-
-import java.util.List;
-
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.provenance.ProvenanceEventRecord;
-import org.apache.nifi.provenance.ProvenanceEventType;
-
-public class EventNode implements ProvenanceEventLineageNode {
-
-    private final ProvenanceEventRecord record;
-    private String clusterNodeIdentifier = null;
-
-    public EventNode(final ProvenanceEventRecord event) {
-        this.record = event;
-    }
-
-    @Override
-    public String getIdentifier() {
-        return String.valueOf(getEventIdentifier());
-    }
-
-    @Override
-    public String getClusterNodeIdentifier() {
-        return clusterNodeIdentifier;
-    }
-
-    public void setClusterNodeIdentifier(final String nodeIdentifier) {
-        this.clusterNodeIdentifier = nodeIdentifier;
-    }
-
-    @Override
-    public LineageNodeType getNodeType() {
-        return LineageNodeType.PROVENANCE_EVENT_NODE;
-    }
-
-    @Override
-    public ProvenanceEventType getEventType() {
-        return record.getEventType();
-    }
-
-    @Override
-    public long getTimestamp() {
-        return record.getEventTime();
-    }
-
-    @Override
-    public long getEventIdentifier() {
-        return record.getEventId();
-    }
-
-    @Override
-    public String getFlowFileUuid() {
-        return record.getAttributes().get(CoreAttributes.UUID.key());
-    }
-
-    @Override
-    public List<String> getParentUuids() {
-        return record.getParentUuids();
-    }
-
-    @Override
-    public List<String> getChildUuids() {
-        return record.getChildUuids();
-    }
-
-    @Override
-    public int hashCode() {
-        return 2938472 + record.hashCode();
-    }
-
-    @Override
-    public boolean equals(final Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (this == obj) {
-            return true;
-        }
-
-        if (!(obj instanceof EventNode)) {
-            return false;
-        }
-
-        final EventNode other = (EventNode) obj;
-        return record.equals(other.record);
-    }
-
-    @Override
-    public String toString() {
-        return "Event[ID=" + record.getEventId() + ", Type=" + record.getEventType() + ", UUID=" + record.getFlowFileUuid() + ", Component=" + record.getComponentId() + "]";
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileLineage.java
----------------------------------------------------------------------
diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileLineage.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileLineage.java
deleted file mode 100644
index c36c38d..0000000
--- a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileLineage.java
+++ /dev/null
@@ -1,76 +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.provenance.lineage;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import static java.util.Objects.requireNonNull;
-
-public class FlowFileLineage implements Lineage {
-
-    private final List<LineageNode> nodes;
-    private final List<LineageEdge> edges;
-
-    public FlowFileLineage(final Collection<LineageNode> nodes, final Collection<LineageEdge> edges) {
-        this.nodes = new ArrayList<>(requireNonNull(nodes));
-        this.edges = new ArrayList<>(requireNonNull(edges));
-    }
-
-    @Override
-    public List<LineageNode> getNodes() {
-        return nodes;
-    }
-
-    @Override
-    public List<LineageEdge> getEdges() {
-        return edges;
-    }
-
-    @Override
-    public int hashCode() {
-        int sum = 923;
-        for (final LineageNode node : nodes) {
-            sum += node.hashCode();
-        }
-
-        for (final LineageEdge edge : edges) {
-            sum += edge.hashCode();
-        }
-
-        return sum;
-    }
-
-    @Override
-    public boolean equals(final Object obj) {
-        if (obj == null) {
-            return false;
-        }
-
-        if (obj == this) {
-            return true;
-        }
-
-        if (!(obj instanceof FlowFileLineage)) {
-            return false;
-        }
-
-        final FlowFileLineage other = (FlowFileLineage) obj;
-        return nodes.equals(other.nodes) && edges.equals(other.edges);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileNode.java
----------------------------------------------------------------------
diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileNode.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileNode.java
deleted file mode 100644
index fdc7470..0000000
--- a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileNode.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.provenance.lineage;
-
-import static java.util.Objects.requireNonNull;
-
-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);
-        this.creationTime = flowFileCreationTime;
-    }
-
-    @Override
-    public String getIdentifier() {
-        return flowFileUuid;
-    }
-
-    @Override
-    public long getTimestamp() {
-        return creationTime;
-    }
-
-    @Override
-    public String getClusterNodeIdentifier() {
-        return clusterNodeIdentifier;
-    }
-
-    @Override
-    public LineageNodeType getNodeType() {
-        return LineageNodeType.FLOWFILE_NODE;
-    }
-
-    @Override
-    public String getFlowFileUuid() {
-        return flowFileUuid;
-    }
-
-    @Override
-    public int hashCode() {
-        return 23498723 + flowFileUuid.hashCode();
-    }
-
-    @Override
-    public boolean equals(final Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (obj == this) {
-            return true;
-        }
-
-        if (!(obj instanceof FlowFileNode)) {
-            return false;
-        }
-
-        final FlowFileNode other = (FlowFileNode) obj;
-        return flowFileUuid.equals(other.flowFileUuid);
-    }
-
-    @Override
-    public String toString() {
-        return "FlowFile[UUID=" + flowFileUuid + "]";
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/flowfile-packager/pom.xml
----------------------------------------------------------------------
diff --git a/commons/flowfile-packager/pom.xml b/commons/flowfile-packager/pom.xml
deleted file mode 100644
index 9cf4ba8..0000000
--- a/commons/flowfile-packager/pom.xml
+++ /dev/null
@@ -1,41 +0,0 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <!--
-      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.
-    -->
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-commons-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-
-    <artifactId>flowfile-packager</artifactId>
-    <version>0.0.1-SNAPSHOT</version>
-    <packaging>jar</packaging>
-
-    <name>FlowFile Packager</name>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-compress</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-lang3</artifactId>
-        </dependency>
-    </dependencies>
-
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackager.java
----------------------------------------------------------------------
diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackager.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackager.java
deleted file mode 100644
index ae16f99..0000000
--- a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackager.java
+++ /dev/null
@@ -1,28 +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.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Map;
-
-public interface FlowFilePackager {
-
-    void packageFlowFile(InputStream in, OutputStream out, Map<String, String> attributes, long fileSize) throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV1.java
----------------------------------------------------------------------
diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV1.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV1.java
deleted file mode 100644
index 07baab1..0000000
--- a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV1.java
+++ /dev/null
@@ -1,104 +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.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.charset.StandardCharsets;
-import java.util.Map;
-
-import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
-import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
-import org.apache.commons.lang3.StringEscapeUtils;
-
-public class FlowFilePackagerV1 implements FlowFilePackager {
-
-    public static final String FILENAME_ATTRIBUTES = "flowfile.attributes";
-    public static final String FILENAME_CONTENT = "flowfile.content";
-    public static final int DEFAULT_TAR_PERMISSIONS = 0644;
-
-    private final int tarPermissions;
-
-    public FlowFilePackagerV1() {
-        this(DEFAULT_TAR_PERMISSIONS);
-    }
-
-    public FlowFilePackagerV1(final int tarPermissions) {
-        this.tarPermissions = tarPermissions;
-    }
-
-    @Override
-    public void packageFlowFile(final InputStream in, final OutputStream out, final Map<String, String> attributes, final long fileSize) throws IOException {
-        try (final TarArchiveOutputStream tout = new TarArchiveOutputStream(out)) {
-            writeAttributesEntry(attributes, tout);
-            writeContentEntry(tout, in, fileSize);
-            tout.finish();
-            tout.flush();
-            tout.close();
-        }
-    }
-
-    private void writeAttributesEntry(final Map<String, String> attributes, final TarArchiveOutputStream tout) throws IOException {
-        final StringBuilder sb = new StringBuilder();
-        sb.append("<?xml version=\"1.0\" encoding=\"UTF-8\"?><!DOCTYPE properties\n  SYSTEM \"http://java.sun.com/dtd/properties.dtd\">\n");
-        sb.append("<properties>");
-        for (final Map.Entry<String, String> entry : attributes.entrySet()) {
-            final String escapedKey = StringEscapeUtils.escapeXml11(entry.getKey());
-            final String escapedValue = StringEscapeUtils.escapeXml11(entry.getValue());
-            sb.append("\n  <entry key=\"").append(escapedKey).append("\">").append(escapedValue).append("</entry>");
-        }
-        sb.append("</properties>");
-
-        final byte[] metaBytes = sb.toString().getBytes(StandardCharsets.UTF_8);
-        final TarArchiveEntry attribEntry = new TarArchiveEntry(FILENAME_ATTRIBUTES);
-        attribEntry.setMode(tarPermissions);
-        attribEntry.setSize(metaBytes.length);
-        tout.putArchiveEntry(attribEntry);
-        tout.write(metaBytes);
-        tout.closeArchiveEntry();
-    }
-
-    private void writeContentEntry(final TarArchiveOutputStream tarOut, final InputStream inStream, final long fileSize) throws IOException {
-        final TarArchiveEntry entry = new TarArchiveEntry(FILENAME_CONTENT);
-        entry.setMode(tarPermissions);
-        entry.setSize(fileSize);
-        tarOut.putArchiveEntry(entry);
-        final byte[] buffer = new byte[512 << 10];//512KB            
-        int bytesRead = 0;
-        while ((bytesRead = inStream.read(buffer)) != -1) { //still more data to read
-            if (bytesRead > 0) {
-                tarOut.write(buffer, 0, bytesRead);
-            }
-        }
-
-        copy(inStream, tarOut);
-        tarOut.closeArchiveEntry();
-    }
-
-    public static long copy(final InputStream source, final OutputStream destination) throws IOException {
-        final byte[] buffer = new byte[8192];
-        int len;
-        long totalCount = 0L;
-        while ((len = source.read(buffer)) > 0) {
-            destination.write(buffer, 0, len);
-            totalCount += len;
-        }
-        return totalCount;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV2.java
----------------------------------------------------------------------
diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV2.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV2.java
deleted file mode 100644
index 6f9d6b1..0000000
--- a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV2.java
+++ /dev/null
@@ -1,146 +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.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Map;
-
-/**
- * <p>
- * Packages a FlowFile, including both its content and its attributes into a
- * single file that is stream-friendly. The encoding scheme is as such:
- * </p>
- *
- * <pre>
- * Length Field : indicates the number of Flow File Attributes in the stream
- * 1 to N times (N=number of Flow File Attributes):
- *      String Field : Flow File Attribute key name
- *      String Field : Flow File Attribute value
- * Long : 8 bytes indicating the length of the Flow File content
- * Content : The next M bytes are the content of the Flow File.
- * </pre>
- *
- * <pre>
- * Encoding of String Field is as follows:
- *      Length Field : indicates the length of the String
- *      1 to N bytes (N=String length, determined by previous field, as described above) : The UTF-8 encoded string value.
- * </pre>
- *
- * <pre>
- * Encoding of Length Field is as follows:
- *      First 2 bytes: Indicate length. If both bytes = 255, this is a special value indicating that the length is
- *                     greater than or equal to 65536 bytes; therefore, the next 4 bytes will indicate the actual length.
- * </pre>
- *
- * <p>
- * Note: All byte-order encoding is Network Byte Order (Most Significant Byte
- * first)
- * </p>
- *
- * <p>
- * The following example shows the bytes expected if we were to encode a
- * FlowFile containing the following attributes where the content is the text
- * "Hello World!":
- *
- * <br><br>
- * Attributes:
- * <pre>
- * +-------+-------+
- * | Key   + Value |
- * + --------------+
- * | A     | a     |
- * + --------------+
- * | B     | b     |
- * + --------------+
- * </pre> Content:<br>
- * Hello World!
- * <br><br>
- * Packaged Byte Encoding (In Hexadecimal Form):
- * <p>
- *
- * <pre>
- * 00 02 00 01 41 00 01 61
- * 00 01 42 00 01 62 00 00
- * 00 00 00 00 00 0C 48 65
- * 6C 6C 6F 20 57 6F 72 6C
- * 64 21
- * </pre>
- */
-public class FlowFilePackagerV2 implements FlowFilePackager {
-
-    private static final int MAX_VALUE_2_BYTES = 65535;
-    private final byte[] writeBuffer = new byte[8];
-
-    @Override
-    public void packageFlowFile(final InputStream in, final OutputStream out, final Map<String, String> attributes, final long fileSize) throws IOException {
-        writeFieldLength(out, attributes.size()); //write out the number of attributes
-        for (final Map.Entry<String, String> entry : attributes.entrySet()) { //write out each attribute key/value pair
-            writeString(entry.getKey(), out);
-            writeString(entry.getValue(), out);
-        }
-        writeLong(out, fileSize);//write out length of data
-        copy(in, out);//write out the actual flow file payload
-    }
-
-    private void copy(final InputStream in, final OutputStream out) throws IOException {
-        final byte[] buffer = new byte[65536];
-        int len;
-        while ((len = in.read(buffer)) > 0) {
-            out.write(buffer, 0, len);
-        }
-    }
-
-    private void writeString(final String val, final OutputStream out) throws IOException {
-        final byte[] bytes = val.getBytes("UTF-8");
-        writeFieldLength(out, bytes.length);
-        out.write(bytes);
-    }
-
-    private void writeFieldLength(final OutputStream out, final int numBytes) throws IOException {
-        // If the value is less than the max value that can be fit into 2 bytes, just use the
-        // actual value. Otherwise, we will set the first 2 bytes to 255/255 and then use the next
-        // 4 bytes to indicate the real length.
-        if (numBytes < MAX_VALUE_2_BYTES) {
-            writeBuffer[0] = (byte) (numBytes >>> 8);
-            writeBuffer[1] = (byte) (numBytes);
-            out.write(writeBuffer, 0, 2);
-        } else {
-            writeBuffer[0] = (byte) 0xff;
-            writeBuffer[1] = (byte) 0xff;
-            writeBuffer[2] = (byte) (numBytes >>> 24);
-            writeBuffer[3] = (byte) (numBytes >>> 16);
-            writeBuffer[4] = (byte) (numBytes >>> 8);
-            writeBuffer[5] = (byte) (numBytes);
-            out.write(writeBuffer, 0, 6);
-        }
-    }
-
-    private void writeLong(final OutputStream out, final long val) throws IOException {
-        writeBuffer[0] = (byte) (val >>> 56);
-        writeBuffer[1] = (byte) (val >>> 48);
-        writeBuffer[2] = (byte) (val >>> 40);
-        writeBuffer[3] = (byte) (val >>> 32);
-        writeBuffer[4] = (byte) (val >>> 24);
-        writeBuffer[5] = (byte) (val >>> 16);
-        writeBuffer[6] = (byte) (val >>> 8);
-        writeBuffer[7] = (byte) (val);
-        out.write(writeBuffer, 0, 8);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV3.java
----------------------------------------------------------------------
diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV3.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV3.java
deleted file mode 100644
index 181f3e3..0000000
--- a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV3.java
+++ /dev/null
@@ -1,93 +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.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Map;
-
-public class FlowFilePackagerV3 implements FlowFilePackager {
-
-    public static final byte[] MAGIC_HEADER = {'N', 'i', 'F', 'i', 'F', 'F', '3'};
-    private static final int MAX_VALUE_2_BYTES = 65535;
-    private final byte[] writeBuffer = new byte[8];
-
-    @Override
-    public void packageFlowFile(final InputStream in, final OutputStream out, final Map<String, String> attributes, final long fileSize) throws IOException {
-        out.write(MAGIC_HEADER);
-
-        if (attributes == null) {
-            writeFieldLength(out, 0);
-        } else {
-            writeFieldLength(out, attributes.size()); //write out the number of attributes
-            for (final Map.Entry<String, String> entry : attributes.entrySet()) { //write out each attribute key/value pair
-                writeString(entry.getKey(), out);
-                writeString(entry.getValue(), out);
-            }
-        }
-
-        writeLong(out, fileSize);//write out length of data
-        copy(in, out);//write out the actual flow file payload
-    }
-
-    private void copy(final InputStream in, final OutputStream out) throws IOException {
-        final byte[] buffer = new byte[65536];
-        int len;
-        while ((len = in.read(buffer)) > 0) {
-            out.write(buffer, 0, len);
-        }
-    }
-
-    private void writeString(final String val, final OutputStream out) throws IOException {
-        final byte[] bytes = val.getBytes("UTF-8");
-        writeFieldLength(out, bytes.length);
-        out.write(bytes);
-    }
-
-    private void writeFieldLength(final OutputStream out, final int numBytes) throws IOException {
-        // If the value is less than the max value that can be fit into 2 bytes, just use the
-        // actual value. Otherwise, we will set the first 2 bytes to 255/255 and then use the next
-        // 4 bytes to indicate the real length.
-        if (numBytes < MAX_VALUE_2_BYTES) {
-            writeBuffer[0] = (byte) (numBytes >>> 8);
-            writeBuffer[1] = (byte) (numBytes);
-            out.write(writeBuffer, 0, 2);
-        } else {
-            writeBuffer[0] = (byte) 0xff;
-            writeBuffer[1] = (byte) 0xff;
-            writeBuffer[2] = (byte) (numBytes >>> 24);
-            writeBuffer[3] = (byte) (numBytes >>> 16);
-            writeBuffer[4] = (byte) (numBytes >>> 8);
-            writeBuffer[5] = (byte) (numBytes);
-            out.write(writeBuffer, 0, 6);
-        }
-    }
-
-    private void writeLong(final OutputStream out, final long val) throws IOException {
-        writeBuffer[0] = (byte) (val >>> 56);
-        writeBuffer[1] = (byte) (val >>> 48);
-        writeBuffer[2] = (byte) (val >>> 40);
-        writeBuffer[3] = (byte) (val >>> 32);
-        writeBuffer[4] = (byte) (val >>> 24);
-        writeBuffer[5] = (byte) (val >>> 16);
-        writeBuffer[6] = (byte) (val >>> 8);
-        writeBuffer[7] = (byte) (val);
-        out.write(writeBuffer, 0, 8);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackager.java
----------------------------------------------------------------------
diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackager.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackager.java
deleted file mode 100644
index fd9d92d..0000000
--- a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackager.java
+++ /dev/null
@@ -1,30 +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.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Map;
-
-public interface FlowFileUnpackager {
-
-    Map<String, String> unpackageFlowFile(InputStream in, OutputStream out) throws IOException;
-
-    boolean hasMoreData() throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV1.java
----------------------------------------------------------------------
diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV1.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV1.java
deleted file mode 100644
index f8ef3d1..0000000
--- a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV1.java
+++ /dev/null
@@ -1,155 +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.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
-import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
-
-public class FlowFileUnpackagerV1 implements FlowFileUnpackager {
-
-    private int flowFilesRead = 0;
-
-    @Override
-    public Map<String, String> unpackageFlowFile(final InputStream in, final OutputStream out) throws IOException {
-        flowFilesRead++;
-        final TarArchiveInputStream tarIn = new TarArchiveInputStream(in);
-        final TarArchiveEntry attribEntry = tarIn.getNextTarEntry();
-        if (attribEntry == null) {
-            return null;
-        }
-
-        final Map<String, String> attributes;
-        if (attribEntry.getName().equals(FlowFilePackagerV1.FILENAME_ATTRIBUTES)) {
-            attributes = getAttributes(tarIn);
-        } else {
-            throw new IOException("Expected two tar entries: "
-                    + FlowFilePackagerV1.FILENAME_CONTENT + " and "
-                    + FlowFilePackagerV1.FILENAME_ATTRIBUTES);
-        }
-
-        final TarArchiveEntry contentEntry = tarIn.getNextTarEntry();
-
-        if (contentEntry != null && contentEntry.getName().equals(FlowFilePackagerV1.FILENAME_CONTENT)) {
-            final byte[] buffer = new byte[512 << 10];//512KB            
-            int bytesRead = 0;
-            while ((bytesRead = tarIn.read(buffer)) != -1) { //still more data to read
-                if (bytesRead > 0) {
-                    out.write(buffer, 0, bytesRead);
-                }
-            }
-            out.flush();
-        } else {
-            throw new IOException("Expected two tar entries: "
-                    + FlowFilePackagerV1.FILENAME_CONTENT + " and "
-                    + FlowFilePackagerV1.FILENAME_ATTRIBUTES);
-        }
-
-        return attributes;
-    }
-
-    protected Map<String, String> getAttributes(final TarArchiveInputStream stream) throws IOException {
-
-        final Properties props = new Properties();
-        props.loadFromXML(new NonCloseableInputStream(stream));
-
-        final Map<String, String> result = new HashMap<>();
-        for (final Entry<Object, Object> entry : props.entrySet()) {
-            final Object keyObject = entry.getKey();
-            final Object valueObject = entry.getValue();
-            if (!(keyObject instanceof String)) {
-                throw new IOException("Flow file attributes object contains key of type "
-                        + keyObject.getClass().getCanonicalName()
-                        + " but expected java.lang.String");
-            } else if (!(keyObject instanceof String)) {
-                throw new IOException("Flow file attributes object contains value of type "
-                        + keyObject.getClass().getCanonicalName()
-                        + " but expected java.lang.String");
-            }
-
-            final String key = (String) keyObject;
-            final String value = (String) valueObject;
-            result.put(key, value);
-        }
-
-        return result;
-    }
-
-    @Override
-    public boolean hasMoreData() throws IOException {
-        return flowFilesRead == 0;
-    }
-
-    public static final class NonCloseableInputStream extends InputStream {
-
-        final InputStream stream;
-
-        public NonCloseableInputStream(final InputStream stream) {
-            this.stream = stream;
-        }
-
-        @Override
-        public void close() {
-        }
-
-        @Override
-        public int read() throws IOException {
-            return stream.read();
-        }
-
-        @Override
-        public int available() throws IOException {
-            return stream.available();
-        }
-
-        @Override
-        public synchronized void mark(int readlimit) {
-            stream.mark(readlimit);
-        }
-
-        @Override
-        public synchronized void reset() throws IOException {
-            stream.reset();
-        }
-
-        @Override
-        public boolean markSupported() {
-            return stream.markSupported();
-        }
-
-        @Override
-        public long skip(long n) throws IOException {
-            return stream.skip(n);
-        }
-
-        @Override
-        public int read(byte b[], int off, int len) throws IOException {
-            return stream.read(b, off, len);
-        }
-
-        @Override
-        public int read(byte b[]) throws IOException {
-            return stream.read(b);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV2.java
----------------------------------------------------------------------
diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV2.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV2.java
deleted file mode 100644
index 500015f..0000000
--- a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV2.java
+++ /dev/null
@@ -1,143 +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.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.HashMap;
-import java.util.Map;
-
-public class FlowFileUnpackagerV2 implements FlowFileUnpackager {
-
-    private final byte readBuffer[] = new byte[8192];
-    private Map<String, String> nextAttributes = null;
-    private boolean haveReadSomething = false;
-
-    @Override
-    public boolean hasMoreData() throws IOException {
-        return nextAttributes != null || !haveReadSomething;
-    }
-
-    protected Map<String, String> readAttributes(final InputStream in) throws IOException {
-        final Map<String, String> attributes = new HashMap<>();
-        final Integer numAttributes = readFieldLength(in); //read number of attributes
-        if (numAttributes == null) {
-            return null;
-        }
-        if (numAttributes == 0) {
-            throw new IOException("flow files cannot have zero attributes");
-        }
-        for (int i = 0; i < numAttributes; i++) { //read each attribute key/value pair
-            final String key = readString(in);
-            final String value = readString(in);
-            attributes.put(key, value);
-        }
-
-        return attributes;
-    }
-
-    @Override
-    public Map<String, String> unpackageFlowFile(final InputStream in, final OutputStream out) throws IOException {
-        final Map<String, String> attributes;
-        if (nextAttributes != null) {
-            attributes = nextAttributes;
-        } else {
-            attributes = readAttributes(in);
-        }
-
-        final long expectedNumBytes = readLong(in); // read length of payload
-        copy(in, out, expectedNumBytes); // read payload
-
-        nextAttributes = readAttributes(in);
-        haveReadSomething = true;
-
-        return attributes;
-    }
-
-    protected String readString(final InputStream in) throws IOException {
-        final Integer numBytes = readFieldLength(in);
-        if (numBytes == null) {
-            throw new EOFException();
-        }
-        final byte[] bytes = new byte[numBytes];
-        fillBuffer(in, bytes, numBytes);
-        return new String(bytes, "UTF-8");
-    }
-
-    private void fillBuffer(final InputStream in, final byte[] buffer, final int length) throws IOException {
-        int bytesRead;
-        int totalBytesRead = 0;
-        while ((bytesRead = in.read(buffer, totalBytesRead, length - totalBytesRead)) > 0) {
-            totalBytesRead += bytesRead;
-        }
-        if (totalBytesRead != length) {
-            throw new EOFException();
-        }
-    }
-
-    protected long copy(final InputStream in, final OutputStream out, final long numBytes) throws IOException {
-        int bytesRead;
-        long totalBytesRead = 0L;
-        while ((bytesRead = in.read(readBuffer, 0, (int) Math.min(readBuffer.length, numBytes - totalBytesRead))) > 0) {
-            out.write(readBuffer, 0, bytesRead);
-            totalBytesRead += bytesRead;
-        }
-
-        if (totalBytesRead < numBytes) {
-            throw new EOFException("Expected " + numBytes + " but received " + totalBytesRead);
-        }
-
-        return totalBytesRead;
-    }
-
-    protected long readLong(final InputStream in) throws IOException {
-        fillBuffer(in, readBuffer, 8);
-        return (((long) readBuffer[0] << 56)
-                + ((long) (readBuffer[1] & 255) << 48)
-                + ((long) (readBuffer[2] & 255) << 40)
-                + ((long) (readBuffer[3] & 255) << 32)
-                + ((long) (readBuffer[4] & 255) << 24)
-                + ((readBuffer[5] & 255) << 16)
-                + ((readBuffer[6] & 255) << 8)
-                + ((readBuffer[7] & 255)));
-    }
-
-    private Integer readFieldLength(final InputStream in) throws IOException {
-        final int firstValue = in.read();
-        final int secondValue = in.read();
-        if (firstValue < 0) {
-            return null;
-        }
-        if (secondValue < 0) {
-            throw new EOFException();
-        }
-        if (firstValue == 0xff && secondValue == 0xff) {
-            int ch1 = in.read();
-            int ch2 = in.read();
-            int ch3 = in.read();
-            int ch4 = in.read();
-            if ((ch1 | ch2 | ch3 | ch4) < 0) {
-                throw new EOFException();
-            }
-            return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4));
-        } else {
-            return ((firstValue << 8) + (secondValue));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV3.java
----------------------------------------------------------------------
diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV3.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV3.java
deleted file mode 100644
index f937585..0000000
--- a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV3.java
+++ /dev/null
@@ -1,161 +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.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-
-public class FlowFileUnpackagerV3 implements FlowFileUnpackager {
-
-    private byte[] nextHeader = null;
-    private boolean haveReadSomething = false;
-    private final byte readBuffer[] = new byte[8192];
-
-    @Override
-    public boolean hasMoreData() throws IOException {
-        return nextHeader != null || !haveReadSomething;
-    }
-
-    private byte[] readHeader(final InputStream in) throws IOException {
-        final byte[] header = new byte[FlowFilePackagerV3.MAGIC_HEADER.length];
-        for (int i = 0; i < header.length; i++) {
-            final int next = in.read();
-            if (next < 0) {
-                if (i == 0) {
-                    return null;
-                }
-
-                throw new IOException("Not in FlowFile-v3 format");
-            }
-            header[i] = (byte) (next & 0xFF);
-        }
-
-        return header;
-    }
-
-    @Override
-    public Map<String, String> unpackageFlowFile(final InputStream in, final OutputStream out) throws IOException {
-        final byte[] header = (nextHeader == null) ? readHeader(in) : nextHeader;
-        if (!Arrays.equals(header, FlowFilePackagerV3.MAGIC_HEADER)) {
-            throw new IOException("Not in FlowFile-v3 format");
-        }
-
-        final Map<String, String> attributes = readAttributes(in);
-        final long expectedNumBytes = readLong(in); // read length of payload
-        copy(in, out, expectedNumBytes); // read payload
-
-        nextHeader = readHeader(in);
-        haveReadSomething = true;
-
-        return attributes;
-    }
-
-    protected Map<String, String> readAttributes(final InputStream in) throws IOException {
-        final Map<String, String> attributes = new HashMap<>();
-        final Integer numAttributes = readFieldLength(in); //read number of attributes
-        if (numAttributes == null) {
-            return null;
-        }
-        if (numAttributes == 0) {
-            throw new IOException("flow files cannot have zero attributes");
-        }
-        for (int i = 0; i < numAttributes; i++) { //read each attribute key/value pair
-            final String key = readString(in);
-            final String value = readString(in);
-            attributes.put(key, value);
-        }
-
-        return attributes;
-    }
-
-    protected String readString(final InputStream in) throws IOException {
-        final Integer numBytes = readFieldLength(in);
-        if (numBytes == null) {
-            throw new EOFException();
-        }
-        final byte[] bytes = new byte[numBytes];
-        fillBuffer(in, bytes, numBytes);
-        return new String(bytes, "UTF-8");
-    }
-
-    private void fillBuffer(final InputStream in, final byte[] buffer, final int length) throws IOException {
-        int bytesRead;
-        int totalBytesRead = 0;
-        while ((bytesRead = in.read(buffer, totalBytesRead, length - totalBytesRead)) > 0) {
-            totalBytesRead += bytesRead;
-        }
-        if (totalBytesRead != length) {
-            throw new EOFException();
-        }
-    }
-
-    protected long copy(final InputStream in, final OutputStream out, final long numBytes) throws IOException {
-        int bytesRead;
-        long totalBytesRead = 0L;
-        while ((bytesRead = in.read(readBuffer, 0, (int) Math.min(readBuffer.length, numBytes - totalBytesRead))) > 0) {
-            out.write(readBuffer, 0, bytesRead);
-            totalBytesRead += bytesRead;
-        }
-
-        if (totalBytesRead < numBytes) {
-            throw new EOFException("Expected " + numBytes + " but received " + totalBytesRead);
-        }
-
-        return totalBytesRead;
-    }
-
-    protected long readLong(final InputStream in) throws IOException {
-        fillBuffer(in, readBuffer, 8);
-        return (((long) readBuffer[0] << 56)
-                + ((long) (readBuffer[1] & 255) << 48)
-                + ((long) (readBuffer[2] & 255) << 40)
-                + ((long) (readBuffer[3] & 255) << 32)
-                + ((long) (readBuffer[4] & 255) << 24)
-                + ((readBuffer[5] & 255) << 16)
-                + ((readBuffer[6] & 255) << 8)
-                + ((readBuffer[7] & 255)));
-    }
-
-    private Integer readFieldLength(final InputStream in) throws IOException {
-        final int firstValue = in.read();
-        final int secondValue = in.read();
-        if (firstValue < 0) {
-            return null;
-        }
-        if (secondValue < 0) {
-            throw new EOFException();
-        }
-        if (firstValue == 0xff && secondValue == 0xff) {
-            int ch1 = in.read();
-            int ch2 = in.read();
-            int ch3 = in.read();
-            int ch4 = in.read();
-            if ((ch1 | ch2 | ch3 | ch4) < 0) {
-                throw new EOFException();
-            }
-            return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4));
-        } else {
-            return ((firstValue << 8) + (secondValue));
-        }
-    }
-
-}


[33/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/OutputStreamHandler.java
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/OutputStreamHandler.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/OutputStreamHandler.java
deleted file mode 100644
index d879722..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/OutputStreamHandler.java
+++ /dev/null
@@ -1,24 +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.scripting;
-
-import java.io.OutputStream;
-
-public interface OutputStreamHandler {
-
-    void write(OutputStream out);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ReaderScript.java
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ReaderScript.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ReaderScript.java
deleted file mode 100644
index b1d89c0..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ReaderScript.java
+++ /dev/null
@@ -1,79 +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.scripting;
-
-import java.io.InputStream;
-import java.util.Map;
-
-import javax.script.Invocable;
-import javax.script.ScriptException;
-
-import org.apache.nifi.processor.Relationship;
-
-/**
- * <p>
- * Script authors should extend this class if they want to follow the "reader"
- * paradigm for NiFi processors.
- * </p>
- *
- * <p>
- * User scripts should implement {@link #route(InputStream)}. <code>route</code>
- * uses a returned relationship name to determine where FlowFiles go. Scripts
- * may also implement {@link #getProcessorRelationships()} to specify available
- * relationship names.
- * </p>
- *
- */
-public class ReaderScript extends Script {
-
-    private Object routeCallback;
-
-    public ReaderScript(Object... callbacks) {
-        super(callbacks);
-        for (Object callback : callbacks) {
-            if (callback instanceof Map<?, ?>) {
-                routeCallback = routeCallback == null && ((Map<?, ?>) callback).containsKey("route") ? callback : routeCallback;
-            }
-        }
-    }
-
-    public ReaderScript() {
-
-    }
-
-    // Simple helper
-    public void process(InputStream input) throws NoSuchMethodException, ScriptException {
-        lastRoute = route(input);
-    }
-
-    /**
-     * Subclasses should examine the provided inputstream, then determine which
-     * relationship the file will be sent down and return its name.
-     *
-     *
-     * @param in a Java InputStream containing the incoming FlowFile.
-     * @return a relationship name
-     * @throws ScriptException
-     * @throws NoSuchMethodException
-     */
-    public Relationship route(InputStream in) throws NoSuchMethodException, ScriptException {
-        Relationship relationship = null;
-        Invocable invocable = (Invocable) this.engine;
-        relationship = (Relationship) invocable.invokeMethod(routeCallback, "route", in);
-        return relationship;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/Script.java
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/Script.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/Script.java
deleted file mode 100644
index 786f541..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/Script.java
+++ /dev/null
@@ -1,303 +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.scripting;
-
-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 javax.script.Invocable;
-import javax.script.ScriptEngine;
-import javax.script.ScriptException;
-
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.logging.ProcessorLog;
-import org.apache.nifi.processor.Relationship;
-
-/**
- * <p>
- * Base class for all scripts. In this framework, only ScriptEngines that
- * implement javax.script.Invocable are supported.
- *
- * </p>
- *
- */
-public class Script {
-
-    public static final Relationship SUCCESS_RELATIONSHIP = new Relationship.Builder()
-            .name("success")
-            .description("Destination of successfully created flow files")
-            .build();
-    public static final Relationship FAIL_RELATIONSHIP = new Relationship.Builder()
-            .name("failure")
-            .description("Destination of flow files when a error occurs in the script")
-            .build();
-
-    static final Set<Relationship> RELATIONSHIPS;
-
-    static {
-        Set<Relationship> rels = new HashSet<>();
-        rels.add(FAIL_RELATIONSHIP);
-        rels.add(SUCCESS_RELATIONSHIP);
-        RELATIONSHIPS = Collections.unmodifiableSet(rels);
-    }
-
-    FlowFile flowFile = null;
-    ScriptEngine engine = null;
-
-    protected Map<String, String> properties = new HashMap<>();
-    protected Relationship lastRoute = SUCCESS_RELATIONSHIP;
-    protected ProcessorLog logger;
-    protected String scriptFileName;
-    protected Map<String, String> attributes = new HashMap<>();
-    protected long flowFileSize = 0;
-    protected long flowFileEntryDate = System.currentTimeMillis();
-
-    // the following are needed due to an inadequate JavaScript ScriptEngine. It will not allow
-    // subclassing a Java Class, only implementing a Java Interface. So, the syntax of JavaScript
-    // scripts looks like subclassing, but actually is just constructing a Script instance and
-    // passing in functions as args to the constructor. When we move to Nashorn JavaScript ScriptEngine
-    // in Java 8, we can get rid of these and revert the subclasses of this class to abstract.
-    protected Object propDescCallback;
-    protected Object relationshipsCallback;
-    protected Object validateCallback;
-    protected Object exceptionRouteCallback;
-
-    /**
-     * Create a Script without any parameters
-     */
-    public Script() {
-    }
-
-    public Script(Object... callbacks) {
-        for (Object callback : callbacks) {
-            if (callback instanceof Map<?, ?>) {
-                propDescCallback = propDescCallback == null && ((Map<?, ?>) callback).containsKey("getPropertyDescriptors") ? callback
-                        : propDescCallback;
-                relationshipsCallback = relationshipsCallback == null && ((Map<?, ?>) callback).containsKey("getRelationships") ? callback
-                        : relationshipsCallback;
-                validateCallback = validateCallback == null && ((Map<?, ?>) callback).containsKey("validate") ? callback : validateCallback;
-                exceptionRouteCallback = exceptionRouteCallback == null && ((Map<?, ?>) callback).containsKey("getExceptionRoute") ? callback
-                        : exceptionRouteCallback;
-            }
-        }
-    }
-
-    /**
-     * Specify a set of properties with corresponding NiFi validators.
-     *
-     * Subclasses that do not override this method will still have access to all
-     * properties via the "properties" field
-     *
-     * @return a list of PropertyDescriptors
-     * @throws ScriptException
-     * @throws NoSuchMethodException
-     */
-    @SuppressWarnings("unchecked")
-    public List<PropertyDescriptor> getPropertyDescriptors() throws NoSuchMethodException, ScriptException {
-        if (propDescCallback != null) {
-            return (List<PropertyDescriptor>) ((Invocable) engine).invokeMethod(propDescCallback, "getPropertyDescriptors", (Object) null);
-        }
-        return Collections.emptyList();
-    }
-
-    /**
-     * Specify a set of reasons why this processor should be invalid.
-     *
-     * Subclasses that do not override this method will depend only on
-     * individual property validators as specified in
-     * {@link #getPropertyDescriptors()}.
-     *
-     * @return a Collection of messages to display to the user, or an empty
-     * Collection if the processor configuration is OK.
-     * @throws ScriptException
-     * @throws NoSuchMethodException
-     */
-    @SuppressWarnings("unchecked")
-    public Collection<String> validate() throws NoSuchMethodException, ScriptException {
-        if (validateCallback != null) {
-            return (Collection<String>) ((Invocable) engine).invokeMethod(validateCallback, "validate", (Object) null);
-        }
-        return Collections.emptyList();
-    }
-
-    void setFlowFile(FlowFile ff) {
-        flowFile = ff;
-        if (null != ff) {
-            // have to clone because ff.getAttributes is unmodifiable
-            this.attributes = new HashMap<>(ff.getAttributes());
-            this.flowFileSize = ff.getSize();
-            this.flowFileEntryDate = ff.getEntryDate();
-        }
-    }
-
-    void setProperties(Map<String, String> map) {
-        properties = new HashMap<>(map);
-    }
-
-    /**
-     * Required to access entire properties map -- Jython (at least) won't let
-     * you read the member variable without a getter
-     *
-     * @return entire parameter map
-     */
-    // change back to protected when we get nashorn
-    public Map<String, String> getProperties() {
-        return properties;
-    }
-
-    /**
-     * Get the named parameter. Some scripting languages make a method call
-     * easier than accessing a member field, so this is a convenience method to
-     * look up values in the properties field.
-     *
-     * @param key a hash key
-     * @return the value pointed at by the key specified
-     */
-    public String getProperty(String key) {
-        return properties.get(key);
-    }
-
-    /**
-     * Name the various relationships by which a file can leave this processor.
-     * Subclasses may override this method to change available relationships.
-     *
-     * @return a collection of relationship names
-     * @throws ScriptException
-     * @throws NoSuchMethodException
-     */
-    @SuppressWarnings("unchecked")
-    public Collection<Relationship> getRelationships() throws NoSuchMethodException, ScriptException {
-        if (relationshipsCallback != null) {
-            return (Collection<Relationship>) ((Invocable) engine).invokeMethod(relationshipsCallback, "getRelationships", (Object) null);
-        }
-        return RELATIONSHIPS;
-    }
-
-    /**
-     * Determine what do with a file that has just been processed.
-     *
-     * After a script runs its "read" or "write" method, it should update the
-     * "lastRoute" field to specify the relationship to which the resulting file
-     * will be sent.
-     *
-     * @return a relationship name
-     */
-    public Relationship getRoute() {
-        return lastRoute;
-    }
-
-    // Required because of a potential issue in Rhino -- protected methods are visible in
-    // subclasses but protected fields (like "lastRoute") are not
-    // change back to protected when we get nashorn
-    public void setRoute(Relationship route) {
-        lastRoute = route;
-    }
-
-    /**
-     * Determine where to send a file if an exception is thrown during
-     * processing.
-     *
-     * Subclasses may override this method to use a different relationship, or
-     * to determine the relationship dynamically. Returning null causes the file
-     * to be deleted instead.
-     *
-     * Defaults to "failure".
-     *
-     * @return the name of the relationship to use in event of an exception, or
-     * null to delete the file.
-     * @throws ScriptException
-     * @throws NoSuchMethodException
-     */
-    public Relationship getExceptionRoute() throws NoSuchMethodException, ScriptException {
-        if (exceptionRouteCallback != null) {
-            return (Relationship) ((Invocable) engine).invokeMethod(exceptionRouteCallback, "getExceptionRoute", (Object) null);
-        }
-        return FAIL_RELATIONSHIP;
-    }
-
-    /*
-     * Some scripting languages make a method call easier than accessing a member field, so this is a convenience method to get
-     * the incoming flow file size.
-     */
-    // Change back to protected when we get nashorn
-    public long getFlowFileSize() {
-        return flowFileSize;
-    }
-
-    /*
-     * Some scripting languages make a method call easier than accessing a member field, so this is a convenience method to get
-     * entry date of the flow file.
-     */
-    // Change back to protected when we get nashorn
-    public long getFlowFileEntryDate() {
-        return flowFileEntryDate;
-    }
-
-    void setLogger(ProcessorLog logger) {
-        this.logger = logger;
-    }
-
-    /*
-     * Required so that scripts in some languages can read access the attribute. Jython (at least) won't let you read the member
-     * variable without a getter
-     */
-    protected ProcessorLog getLogger() {
-        return this.logger;
-    }
-
-    void setFileName(String scriptFileName) {
-        this.scriptFileName = scriptFileName;
-    }
-
-    public String getFileName() {
-        return this.scriptFileName;
-    }
-
-    // this one's public because it's needed by ExecuteScript to update the flow file's attributes AFTER processing is done
-    public Map<String, String> getAttributes() {
-        return this.attributes;
-    }
-
-    /*
-     * Some scripting languages make a method call easier than accessing a member field, so this is a convenience method to look
-     * up values in the attributes field.
-     */
-    // Change back to protected when we get nashorn
-    public String getAttribute(String key) {
-        return this.attributes.get(key);
-    }
-
-    /*
-     * Some scripting languages make a method call easier than accessing a member field, so this is a convenience method to set
-     * key/value pairs in the attributes field.
-     */
-    // Change back to protected when we get nashorn
-    public void setAttribute(String key, String value) {
-        this.attributes.put(key, value);
-    }
-
-    void setEngine(ScriptEngine scriptEngine) {
-        this.engine = scriptEngine;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ScriptEngineFactory.java
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ScriptEngineFactory.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ScriptEngineFactory.java
deleted file mode 100644
index 6f38886..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ScriptEngineFactory.java
+++ /dev/null
@@ -1,117 +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.scripting;
-
-import java.io.File;
-import java.util.concurrent.ConcurrentHashMap;
-
-import javax.script.ScriptEngine;
-import javax.script.ScriptEngineManager;
-import javax.script.ScriptException;
-
-import org.apache.commons.lang3.StringUtils;
-import org.jruby.embed.PropertyName;
-
-public class ScriptEngineFactory {
-
-    private static final String THREADING = "THREADING";
-    private static final String MULTITHREADED = "MULTITHREADED";
-    private static final String STATELESS = "STATELESS";
-    private static final String THREAD_ISOLATED = "THREAD-ISOLATED";
-    final static ScriptEngineManager scriptEngMgr;
-
-    static {
-        System.setProperty(PropertyName.LOCALCONTEXT_SCOPE.toString(), "singlethread");
-        System.setProperty(PropertyName.COMPILEMODE.toString(), "jit");
-        System.setProperty(PropertyName.COMPATVERSION.toString(), "JRuby1.9");
-        System.setProperty(PropertyName.LOCALVARIABLE_BEHAVIOR.toString(), "transient");
-        System.setProperty("compile.invokedynamic", "false");
-        System.setProperty(PropertyName.LAZINESS.toString(), "true");
-        scriptEngMgr = new ScriptEngineManager();
-    }
-    final ConcurrentHashMap<String, ScriptEngine> threadSafeEngines = new ConcurrentHashMap<>();
-
-    ScriptEngine getEngine(String extension) {
-        ScriptEngine engine = threadSafeEngines.get(extension);
-        if (null == engine) {
-            engine = scriptEngMgr.getEngineByExtension(extension);
-            if (null == engine) {
-                throw new IllegalArgumentException("No ScriptEngine exists for extension " + extension);
-            }
-
-            Object threading = engine.getFactory().getParameter(THREADING);
-            // the MULTITHREADED status means that the scripts need to be careful about sharing state
-            if (THREAD_ISOLATED.equals(threading) || STATELESS.equals(threading) || MULTITHREADED.equals(threading)) {
-                ScriptEngine cachedEngine = threadSafeEngines.putIfAbsent(extension, engine);
-                if (null != cachedEngine) {
-                    engine = cachedEngine;
-                }
-            }
-        }
-        return engine;
-    }
-
-    ScriptEngine getNewEngine(File scriptFile, String extension) throws ScriptException {
-        ScriptEngine engine = scriptEngMgr.getEngineByExtension(extension);
-        if (null == engine) {
-            throw new IllegalArgumentException("No ScriptEngine exists for extension " + extension);
-        }
-        // Initialize some paths
-        StringBuilder sb = new StringBuilder();
-        switch (extension) {
-            case "rb":
-                String parent = scriptFile.getParent();
-                parent = StringUtils.replace(parent, "\\", "/");
-                sb.append("$:.unshift '")
-                        .append(parent)
-                        .append("'\n")
-                        .append("$:.unshift File.join '")
-                        .append(parent)
-                        .append("', 'lib'\n");
-                engine.eval(sb.toString());
-
-                break;
-            case "py":
-                parent = scriptFile.getParent();
-                parent = StringUtils.replace(parent, "\\", "/");
-                String lib = parent + "/lib";
-                sb.append("import sys\n").append("sys.path.append('").append(parent)
-                        .append("')\n").append("sys.path.append('")
-                        .append(lib)
-                        .append("')\n")
-                        .append("__file__ = '")
-                        .append(scriptFile.getAbsolutePath())
-                        .append("'\n");
-                engine.eval(sb.toString());
-                break;
-            default:
-                break;
-        }
-
-        Object threading = engine.getFactory().getParameter(THREADING);
-        // the MULTITHREADED status means that the scripts need to be careful about sharing state
-        if (THREAD_ISOLATED.equals(threading) || STATELESS.equals(threading) || MULTITHREADED.equals(threading)) {
-            // replace prior instance if any
-            threadSafeEngines.put(extension, engine);
-        }
-        return engine;
-    }
-
-    boolean isThreadSafe(String scriptExtension) {
-        return threadSafeEngines.containsKey(scriptExtension);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ScriptFactory.java
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ScriptFactory.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ScriptFactory.java
deleted file mode 100644
index da18606..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ScriptFactory.java
+++ /dev/null
@@ -1,269 +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.scripting;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.security.DigestInputStream;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
-
-import javax.script.Bindings;
-import javax.script.Compilable;
-import javax.script.CompiledScript;
-import javax.script.ScriptContext;
-import javax.script.ScriptEngine;
-import javax.script.ScriptException;
-import javax.script.SimpleBindings;
-
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.io.BufferedInputStream;
-import org.apache.nifi.logging.ProcessorLog;
-
-import org.apache.commons.io.FileUtils;
-
-/**
- * While this is a 'factory', it is not a singleton because we want a factory
- * per processor. This factory has state, all of which belong to only one
- * processor.
- *
- */
-public class ScriptFactory {
-
-    private final ScriptEngineFactory engineFactory = new ScriptEngineFactory();
-    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
-    private final ReadLock readLock = lock.readLock();
-    private final WriteLock writeLock = lock.writeLock();
-    private final ProcessorLog logger;
-
-    private volatile CompiledScript compiledScript;
-    private volatile String scriptText;
-    private volatile byte[] md5Hash;
-    private volatile long lastTimeChecked;
-    private volatile String scriptFileName;
-    private volatile long scriptCheckIntervalMS = 15000;
-
-    public ScriptFactory(ProcessorLog logger) {
-        this.logger = logger;
-    }
-
-    public void setScriptCheckIntervalMS(long msecs) {
-        this.scriptCheckIntervalMS = msecs;
-    }
-
-    /**
-     * @param aScriptFileName
-     * @param properties
-     * @param flowFile
-     * @return
-     * @throws IOException
-     * @throws ScriptException
-     */
-    public Script getScript(final String aScriptFileName, final Map<String, String> properties, final FlowFile flowFile)
-            throws IOException, ScriptException {
-        final Script instance;
-        long now = System.currentTimeMillis();
-        readLock.lock();
-        try {
-            if (!aScriptFileName.equals(this.scriptFileName)) {
-                readLock.unlock();
-                writeLock.lock();
-                try {
-                    if (!aScriptFileName.equals(this.scriptFileName)) {
-                        // need to get brand new engine
-                        compiledScript = null;
-                        this.md5Hash = getMD5Hash(aScriptFileName);
-                        this.lastTimeChecked = now;
-                        this.scriptFileName = aScriptFileName;
-                        updateEngine();
-                    } // else another thread beat me to the change...so just get a script
-                } finally {
-                    readLock.lock();
-                    writeLock.unlock();
-                }
-            } else if (lastTimeChecked + scriptCheckIntervalMS < now) {
-                readLock.unlock();
-                writeLock.lock();
-                try {
-                    if (lastTimeChecked + scriptCheckIntervalMS < now) {
-                        byte[] md5 = getMD5Hash(this.scriptFileName);
-                        if (!MessageDigest.isEqual(md5Hash, md5)) {
-                            // need to get brand new engine
-                            compiledScript = null;
-                            updateEngine();
-                            this.md5Hash = md5;
-                        } // else no change to script, so just update time checked
-                        this.lastTimeChecked = now;
-                    } // else another thread beat me to the check...so just get a script
-                } finally {
-                    readLock.lock();
-                    writeLock.unlock();
-                }
-            }
-            try {
-                instance = getScriptInstance(properties);
-                instance.setFileName(this.scriptFileName);
-                instance.setProperties(properties);
-                instance.setLogger(logger);
-                instance.setFlowFile(flowFile);
-            } catch (ScriptException e) {
-                // need to reset state to enable re-initialization
-                this.lastTimeChecked = 0;
-                this.scriptFileName = null;
-                throw e;
-            }
-        } finally {
-            readLock.unlock();
-        }
-
-        return instance;
-
-    }
-
-    public Script getScript(String aScriptFileName) throws ScriptException, IOException {
-        Map<String, String> props = new HashMap<>();
-        return getScript(aScriptFileName, props, null);
-    }
-
-    private byte[] getMD5Hash(String aScriptFileName) throws FileNotFoundException, IOException {
-        byte[] messageDigest = null;
-        try (FileInputStream fis = new FileInputStream(aScriptFileName);
-                DigestInputStream dis = new DigestInputStream(new BufferedInputStream(fis), MessageDigest.getInstance("MD5"))) {
-
-            byte[] bytes = new byte[8192];
-            while (dis.read(bytes) != -1) {
-                // do nothing...just computing the md5 hash
-            }
-            messageDigest = dis.getMessageDigest().digest();
-        } catch (NoSuchAlgorithmException swallow) {
-            // MD5 is a legitimate format
-        }
-        return messageDigest;
-    }
-
-    private String getScriptText(File scriptFile, String extension) throws IOException {
-        final String script;
-        switch (extension) {
-            case "rb":
-                script = JRubyScriptFactory.INSTANCE.getScript(scriptFile);
-                break;
-
-            case "js":
-                script = JavaScriptScriptFactory.INSTANCE.getScript(scriptFile);
-                break;
-
-            case "py":
-                script = JythonScriptFactory.INSTANCE.getScript(scriptFile);
-                break;
-
-            default:
-                script = FileUtils.readFileToString(scriptFile);
-        }
-        return script;
-    }
-
-    private Script getScriptInstance(final Map<String, String> properties) throws ScriptException {
-
-        Map<String, Object> localThreadVariables = new HashMap<>();
-        final String extension = getExtension(scriptFileName);
-        String loggerVariableKey = getVariableName("GLOBAL", "logger", extension);
-        localThreadVariables.put(loggerVariableKey, logger);
-        String propertiesVariableKey = getVariableName("INSTANCE", "properties", extension);
-        localThreadVariables.put(propertiesVariableKey, properties);
-        localThreadVariables.put(ScriptEngine.FILENAME, scriptFileName);
-        final Bindings bindings = new SimpleBindings(localThreadVariables);
-        final ScriptEngine scriptEngine = engineFactory.getEngine(extension);
-        Script instance;
-        if (compiledScript == null) {
-            instance = (Script) scriptEngine.eval(scriptText, bindings);
-            if (instance == null) { // which it will be for python and also for local variables in javascript
-                instance = (Script) scriptEngine.eval("instance", bindings);
-            }
-        } else {
-            instance = (Script) compiledScript.eval(bindings);
-            if (instance == null) { // which it will be for python and also for local variables in javascript
-                instance = (Script) compiledScript.getEngine().eval("instance", bindings);
-            }
-        }
-        instance.setEngine(scriptEngine);
-        return instance;
-    }
-
-    /*
-     * Must have writeLock when calling this!!!!
-     */
-    private void updateEngine() throws IOException, ScriptException {
-        final String extension = getExtension(scriptFileName);
-        // if engine is thread safe, it's being reused...if it's a JrubyEngine it
-        File scriptFile = new File(this.scriptFileName);
-        ScriptEngine scriptEngine = engineFactory.getNewEngine(scriptFile, extension);
-        scriptText = getScriptText(scriptFile, extension);
-        Map<String, Object> localThreadVariables = new HashMap<>();
-        String loggerVariableKey = getVariableName("GLOBAL", "logger", extension);
-        localThreadVariables.put(loggerVariableKey, logger);
-        String propertiesVariableKey = getVariableName("INSTANCE", "properties", extension);
-        localThreadVariables.put(propertiesVariableKey, new HashMap<String, String>());
-        localThreadVariables.put(ScriptEngine.FILENAME, scriptFileName);
-        if (scriptEngine instanceof Compilable) {
-            Bindings bindings = new SimpleBindings(localThreadVariables);
-            scriptEngine.setBindings(bindings, ScriptContext.ENGINE_SCOPE);
-            compiledScript = ((Compilable) scriptEngine).compile(scriptText);
-        }
-        logger.debug("Updating Engine!!");
-    }
-
-    private String getVariableName(String scope, String variableName, String extension) {
-        String result;
-        switch (extension) {
-            case "rb":
-                switch (scope) {
-                    case "GLOBAL":
-                        result = '$' + variableName;
-                        break;
-                    case "INSTANCE":
-                        result = '@' + variableName;
-                        break;
-                    default:
-                        result = variableName;
-                        break;
-                }
-
-                break;
-
-            default:
-                result = variableName;
-                break;
-        }
-        return result;
-    }
-
-    private String getExtension(String aScriptFileName) {
-        int dotPos = aScriptFileName.lastIndexOf('.');
-        if (dotPos < 1) {
-            throw new IllegalArgumentException("Script file name must have an extension");
-        }
-        final String extension = aScriptFileName.substring(dotPos + 1);
-        return extension;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/WriterScript.java
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/WriterScript.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/WriterScript.java
deleted file mode 100644
index 7eef98b..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/WriterScript.java
+++ /dev/null
@@ -1,67 +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.scripting;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Map;
-
-import javax.script.Invocable;
-import javax.script.ScriptException;
-
-/**
- * <p>
- * Script authors should extend this class if they want to follow the
- * "processCallback" paradigm for NiFi processors.
- * </p>
- *
- * <p>
- * At a minimum, scripts must implement
- * <code>process(FileInputStream, FileOutputStream)</code>.
- * </p>
- *
- * <p>
- * By default, all files processed will be sent to the relationship
- * <em>success</em>, unless the scriptFileName raises an exception, in which
- * case the file will be sent to <em>failure</em>. Implement
- * {@link #getProcessorRelationships()} and/or {@link #getRoute()} to change
- * this behavior.
- * </p>
- *
- */
-public class WriterScript extends Script {
-
-    private Object processCallback;
-
-    public WriterScript() {
-
-    }
-
-    public WriterScript(Object... callbacks) {
-        super(callbacks);
-        for (Object callback : callbacks) {
-            if (callback instanceof Map<?, ?>) {
-                processCallback = processCallback == null && ((Map<?, ?>) callback).containsKey("process") ? callback : processCallback;
-            }
-        }
-    }
-
-    public void process(InputStream in, OutputStream out) throws NoSuchMethodException, ScriptException {
-        Invocable inv = (Invocable) engine;
-        inv.invokeMethod(processCallback, "process", in, out);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
deleted file mode 100644
index 20a3982..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ /dev/null
@@ -1,15 +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.
-org.apache.nifi.processors.script.ExecuteScript

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/docs/org.apache.nifi.processors.script.ExecuteScript/index.html
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/docs/org.apache.nifi.processors.script.ExecuteScript/index.html b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/docs/org.apache.nifi.processors.script.ExecuteScript/index.html
deleted file mode 100644
index acb47c5..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/docs/org.apache.nifi.processors.script.ExecuteScript/index.html
+++ /dev/null
@@ -1,264 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-    <!--
-      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.
-    -->
-    <head>
-        <meta charset="utf-8" />
-        <title>ExecuteScript</title>
-
-        <link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
-    </head>
-
-    <body>
-        <!-- Processor Documentation ================================================== -->
-        <h2>Description:</h2>
-        <p>
-            This processor provides the capability to execute scripts in various scripting languages, and passes into the scripts 
-            the input stream and output stream(s) representing an incoming flow file and any created flow files. The processor is designed to be
-            thread safe, so multiple concurrent tasks may execute against a single script. The processor provides a framework which enables 
-            script writers to implement 3 different types of scripts:
-        <ul>
-            ReaderScript - which enables stream-based reading of a FlowFile's content</br>
-            WriterScript - which enables stream-based reading and writing/modifying of a FlowFile's content</br>
-            ConverterScript - which enables stream-based reading a FlowFile's content and stream-based writing to newly created FlowFiles</br>
-        </ul>
-        Presently, the processor supports 3 scripting languages: Ruby, Python, and JavaScript. The processor is built on the 
-        javax.script API which enables ScriptEngine discovery, thread management, and encapsulates much of the low level bridging-code that 
-        enables Java to Script language integration. Thus, it is designed to be easily extended to other scripting languages. </br>
-        The attributes of a FlowFile and properties of the Processor are exposed to the script by either a variable in the base class or 
-        a getter method. A script may declare new Processor Properties and different Relationships via overriding the getPropertyDescriptors 
-        and getRelationships methods, respectively. 
-    </p>
-    The processor provides some boilerplate script to aid in the creation of the three different types of scripts. For example,
-    the processor provides import statements for classes commonly used within a processor.
-    <pre>
-                'org.apache.nifi.components.PropertyDescriptor'
-                'org.apache.nifi.components.Validator'
-                'org.apache.nifi.processor.util.StandardValidators'
-                'org.apache.nifi.processor.Relationship'
-                'org.apache.nifi.logging.ProcessorLog'
-                'org.apache.nifi.scripting.ReaderScript'
-                'org.apache.nifi.scripting.WriterScript'
-                'org.apache.nifi.scripting.ConverterScript'
-    </pre>
-    The processor appends to the script's execution path the parent directory of the specified script file and a sub-directory
-    called 'lib', which may be useful for supporting scripts. </p>
-<p>
-    <strong>Shared Variables</strong>
-</p>
-The following variables are provided as shared variables for the scripts:
-<ul>
-    <li>logger 
-        <ul>
-            <li> The processor's logger </li>
-            <li> Scope is GLOBAL, thus in Ruby the syntax is $logger</li>
-        </ul>
-    </li>
-    <li>properties
-        <ul>
-            <li> A Map of the processor's configuration properties; key and value are strings</li>
-            <li> Scope is INSTANCE, thus in Ruby the syntax is @properties</li>
-        </ul>
-    </li>
-</ul>
-<p>
-    <strong>Properties:</strong>
-</p>
-<p>
-    In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered 
-    optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language 
-    (or simply, "expression language"), that is also indicated. Of particular note: This processor allows scripts to define additional 
-    Processor properties, which will not be initially visible. Once the processor's configuration is validated, script defined properties
-    will become visible, and may affect the validity of the processor.
-</p>
-<ul>
-    <li>
-        <strong>Script File Name</strong>
-        <ul>
-            <li>Script location, can be relative or absolute path.</li>
-            <li>Default value: no default</li>
-            <li>Supports expression language: false</li>
-        </ul>
-    </li>
-    <li>
-        <strong>Script Check Interval</strong>
-        <ul>
-            <li>The time period between checking for updates to a script.</li>
-            <li>Default value: 15 sec</li>
-            <li>Supports expression language: false</li>
-        </ul>
-    </li>
-</ul>
-
-<p>
-    <strong>Relationships:</strong>
-</p>
-<p>
-    The initial 'out of the box' relationships are below. Of particular note is the ability of a script to change the set of
-    relationships. However, any relationships defined by the script will not be visible until the processor's configuration has been 
-    validated. Once done, new relationships will become visible.
-</p>
-<ul>
-    <li>
-        success
-        <ul>
-            <li>Used when a file is successfully processed by a script.</li>
-        </ul>   
-    </li>
-    <li>
-        failure
-        <ul>
-            <li>Used when an error occurs while processing a file with a script.</li>
-        </ul>   
-    </li>
-</ul>
-
-<p>
-    <strong>Example Scripts:</strong>
-</p>
-<ul>
-    JavaScript example - the 'with' statement imports packages defined in the framework and limits the importing to the local scope, 
-    rather than global. The 'Scripting' variable uses the JavaImporter class within JavaScript. Since the 'instance' variable is intended to 
-    be local scope (not global), it must be named 'instance' as it it not passed back to the processor upon script evaluation and must be 
-    fetched. If you make it global, you can name it whatever you'd like...but this is intended to be multi-threaded so do so at your own 
-    risk.</p>
-Presently, there are issues with the JavaScript scripting engine that prevent sub-classing the base classes in the Processor's Java 
-framework. So, what is actually happening is an instance of the ReaderScript is created with a provided callback object. When we are able
-to move to a more competent scripting engine (supposedly in Java 8), the code below will remain the same, but the 'instance' variable 
-will actually be a sub-class of ReaderScript.
-<pre> 
-              with (Scripting) {
-                var instance = new ReaderScript({
-                    route : function(input) {
-                        var str = IOUtils.toString(input);
-                        var expr = instance.getProperty("expr");
-                        filename = instance.attributes.get("filename");
-                        instance.setAttribute("filename", filename + ".modified");
-                        if (str.match(expr)) {
-                            return Script.FAIL_RELATIONSHIP;
-                        } else {
-                            return Script.SUCCESS_RELATIONSHIP;
-                        }
-                    }
-                });
-              }
-</pre>
-Ruby example - the 'OutputStreamHandler' is an interface which is called when creating flow files.
-<pre>
-                java_import 'org.apache.nifi.scripting.OutputStreamHandler'
-                class SimpleConverter < ConverterScript
-                  field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP, :logger, :attributes
-                  
-                  def convert(input)
-                    in_io = input.to_io
-                    createFlowFile("firstLine", FAIL_RELATIONSHIP, OutputStreamHandler.impl do |method, out|
-                        out_io = out.to_io
-                        out_io << in_io.readline.to_java_bytes
-                        out_io.close
-                        logger.debug("Wrote data to failure...this message logged with logger from super class")
-                      end)
-                      
-                    createFlowFile("otherLines", SUCCESS_RELATIONSHIP, OutputStreamHandler.impl do |method, out|
-                        out_io = out.to_io
-                        in_io.each_line { |line|
-                          out_io << line
-                        }
-                        out_io.close
-                        logger.debug("Wrote data to success...this message logged with logger from super class")
-                      end)
-                    in_io.close
-                  end
-                   
-                end
-                
-                $logger.debug("Creating SimpleConverter...this message logged with logger from shared variables")
-                SimpleConverter.new
-</pre>
-Python example - The difficulty with Python is that it does not return objects upon script evaluation, so the instance of the Script 
-class must be fetched by name. Thus, you must define a variable called 'instance'.
-<pre>
-                import re
-                
-                class RoutingReader(ReaderScript):
-                    A = Relationship.Builder().name("a").description("some good stuff").build()
-                    B = Relationship.Builder().name("b").description("some other stuff").build()
-                    C = Relationship.Builder().name("c").description("some bad stuff").build()
-                    
-                    def getRelationships(self):
-                        return [self.A,self.B,self.C]
-                  
-                    def getExceptionRoute(self):
-                        return self.C
-                  
-                    def route( self, input ):
-                        logger.info("Executing route")
-                        for line in FileUtil.wrap(input):
-                            if re.match("^bad", line, re.IGNORECASE):
-                                return self.B
-                            if re.match("^sed", line):
-                                raise RuntimeError("That's no good!")
-                
-                        return self.A
-                logger.debug("Constructing instance")
-                instance = RoutingReader()
-            
-</pre>
-</ul>
-<p>
-    <strong>Script API:</strong>
-</p>
-<ul>
-    <li>getAttribute(String) : String</li>
-    <li>getAttributes() : Map(String,String)</li>
-    <li>getExceptionRoute() : Relationship</li>
-    <li>getFileName() : String</li>
-    <li>getFlowFileEntryDate() : Calendar</li>
-    <li>getFlowFileSize() : long</li>
-    <li>getProperties() : Map(String, String)</li>
-    <li>getProperty(String) : String</li>
-    <li>getPropertyDescriptors() : List(PropertyDescriptor)</li>
-    <li>getRelationships() : Collection(Relationship)</li>
-    <li>getRoute() : Relationship</li>
-    <li>setRoute(Relationship)</li>
-    <li>setAttribute(String, String)</li>
-    <li>validate() : Collection(String)</li>
-</ul>
-<p>
-    <strong>ReaderScript API:</strong>
-</p>
-<ul>
-    <li>route(InputStream) : Relationship</li>
-</ul>
-<p>
-    <strong>WriterScript API:</strong>
-</p>
-<ul>
-    <li>process(InputStream, OutputStream)</li>
-</ul>
-<p>
-    <strong>ConverterScript API:</strong>
-</p>
-<ul>
-    <li>convert(InputStream)</li>
-    <li>createFlowFile(String, Relationship, OutputStreamHandler)</li>
-</ul>
-<p>
-    <strong>OutputStreamHandler API:</strong>
-</p>
-<ul>
-    <li>write(OutputStream)</li>
-</ul>
-</body>
-</html>


[30/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java
deleted file mode 100644
index 4b89655..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java
+++ /dev/null
@@ -1,1056 +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.admin.dao.impl;
-
-import java.sql.Connection;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.nifi.action.Action;
-import org.apache.nifi.action.Component;
-import org.apache.nifi.action.Operation;
-import org.apache.nifi.action.component.details.ComponentDetails;
-import org.apache.nifi.action.component.details.ProcessorDetails;
-import org.apache.nifi.action.component.details.RemoteProcessGroupDetails;
-import org.apache.nifi.action.details.ActionDetails;
-import org.apache.nifi.action.details.ConfigureDetails;
-import org.apache.nifi.action.details.ConnectDetails;
-import org.apache.nifi.action.details.MoveDetails;
-import org.apache.nifi.action.details.PurgeDetails;
-import org.apache.nifi.admin.RepositoryUtils;
-import org.apache.nifi.admin.dao.ActionDAO;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.history.History;
-import org.apache.nifi.history.HistoryQuery;
-import org.apache.nifi.history.PreviousValue;
-import org.apache.commons.lang3.StringUtils;
-
-/**
- *
- */
-public class StandardActionDAO implements ActionDAO {
-
-    // ------------
-    // action table
-    // ------------
-    private static final String INSERT_ACTION = "INSERT INTO ACTION ("
-            + "USER_DN, USER_NAME, SOURCE_ID, SOURCE_NAME, SOURCE_TYPE, OPERATION, ACTION_TIMESTAMP"
-            + ") VALUES ("
-            + "?, "
-            + "?, "
-            + "?, "
-            + "?, "
-            + "?, "
-            + "?, "
-            + "?, "
-            + ")";
-
-    // -----------------
-    // component details
-    // -----------------
-    private static final String INSERT_PROCESSOR_DETAILS = "INSERT INTO PROCESSOR_DETAILS ("
-            + "ACTION_ID, TYPE"
-            + ") VALUES ("
-            + "?, "
-            + "?"
-            + ")";
-
-    private static final String INSERT_REMOTE_PROCESS_GROUP_DETAILS = "INSERT INTO REMOTE_PROCESS_GROUP_DETAILS ("
-            + "ACTION_ID, URI"
-            + ") VALUES ("
-            + "?, "
-            + "?"
-            + ")";
-
-    // --------------
-    // action details
-    // --------------
-    private static final String INSERT_CONFIGURE_DETAILS = "INSERT INTO CONFIGURE_DETAILS ("
-            + "ACTION_ID, NAME, VALUE, PREVIOUS_VALUE"
-            + ") VALUES ("
-            + "?, "
-            + "?, "
-            + "?, "
-            + "?"
-            + ")";
-
-    private static final String INSERT_CONNECT_DETAILS = "INSERT INTO CONNECT_DETAILS ("
-            + "ACTION_ID, SOURCE_ID, SOURCE_NAME, SOURCE_TYPE, RELATIONSHIP, DESTINATION_ID, DESTINATION_NAME, DESTINATION_TYPE"
-            + ") VALUES ("
-            + "?, "
-            + "?, "
-            + "?, "
-            + "?, "
-            + "?, "
-            + "?, "
-            + "?, "
-            + "?"
-            + ")";
-
-    private static final String INSERT_MOVE_DETAILS = "INSERT INTO MOVE_DETAILS ("
-            + "ACTION_ID, GROUP_ID, GROUP_NAME, PREVIOUS_GROUP_ID, PREVIOUS_GROUP_NAME"
-            + ") VALUES ("
-            + "?, "
-            + "?, "
-            + "?, "
-            + "?, "
-            + "?"
-            + ")";
-
-    private static final String INSERT_PURGE_DETAILS = "INSERT INTO PURGE_DETAILS ("
-            + "ACTION_ID, END_DATE"
-            + ") VALUES ("
-            + "?, "
-            + "?"
-            + ")";
-
-    // ------------
-    // action table
-    // ------------
-    private static final String SELECT_ACTIONS = "SELECT * FROM ACTION";
-
-    private static final String SELECT_ACTION_COUNT = "SELECT COUNT(*) AS ACTION_COUNT FROM ACTION";
-
-    private static final String SELECT_ACTION_BY_ID = "SELECT * "
-            + "FROM ACTION "
-            + "WHERE "
-            + "ID = ?";
-
-    private static final String DELETE_ACTIONS = "DELETE FROM ACTION WHERE ACTION_TIMESTAMP < ?";
-
-    private static final String DELETE_SPECIFIC_ACTIONS = "DELETE FROM %s WHERE %s IN (SELECT ID FROM ACTION WHERE ACTION_TIMESTAMP < ?)";
-
-    // -----------------
-    // component details
-    // -----------------
-    private static final String SELECT_PROCESSOR_DETAILS_FOR_ACTION = "SELECT * FROM PROCESSOR_DETAILS WHERE ACTION_ID = ?";
-
-    private static final String SELECT_REMOTE_PROCESS_GROUP_DETAILS_FOR_ACTION = "SELECT * FROM REMOTE_PROCESS_GROUP_DETAILS WHERE ACTION_ID = ?";
-
-    // --------------
-    // action details
-    // --------------
-    private static final String SELECT_MOVE_DETAILS_FOR_ACTION = "SELECT * FROM MOVE_DETAILS WHERE ACTION_ID = ?";
-
-    private static final String SELECT_CONFIGURE_DETAILS_FOR_ACTION = "SELECT * FROM CONFIGURE_DETAILS WHERE ACTION_ID = ?";
-
-    private static final String SELECT_CONNECT_DETAILS_FOR_ACTION = "SELECT * FROM CONNECT_DETAILS WHERE ACTION_ID = ?";
-
-    private static final String SELECT_PURGE_DETAILS_FOR_ACTION = "SELECT * FROM PURGE_DETAILS WHERE ACTION_ID = ?";
-
-    // ---------------
-    // previous values
-    // ---------------
-    private static final String SELECT_PREVIOUSLY_CONFIGURED_FIELDS = "SELECT DISTINCT CD.NAME "
-            + "FROM CONFIGURE_DETAILS CD "
-            + "INNER JOIN ACTION A "
-            + "ON CD.ACTION_ID = A.ID "
-            + "WHERE A.SOURCE_ID = ?";
-
-    private static final String SELECT_PREVIOUS_VALUES = "SELECT CD.VALUE, "
-            + "A.ACTION_TIMESTAMP, "
-            + "A.USER_NAME "
-            + "FROM CONFIGURE_DETAILS CD "
-            + "INNER JOIN ACTION A "
-            + "ON CD.ACTION_ID = A.ID "
-            + "WHERE A.SOURCE_ID = ? AND CD.NAME = ? "
-            + "ORDER BY A.ACTION_TIMESTAMP DESC "
-            + "LIMIT 4";
-
-    private Connection connection;
-    private Map<String, String> columnMap;
-
-    public StandardActionDAO(Connection connection) {
-        this.connection = connection;
-
-        // initialize the column mappings
-        this.columnMap = new HashMap<>();
-        this.columnMap.put("timestamp", "ACTION_TIMESTAMP");
-        this.columnMap.put("sourceName", "SOURCE_NAME");
-        this.columnMap.put("sourceType", "SOURCE_TYPE");
-        this.columnMap.put("operation", "OPERATION");
-        this.columnMap.put("userName", "USER_NAME");
-    }
-
-    @Override
-    public void createAction(Action action) throws DataAccessException {
-        if (action.getUserDn() == null) {
-            throw new IllegalArgumentException("User cannot be null.");
-        }
-
-        if (action.getTimestamp() == null) {
-            throw new IllegalArgumentException("Action timestamp cannot be null.");
-        }
-
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            // obtain a statement to insert to the action table
-            statement = connection.prepareStatement(INSERT_ACTION, Statement.RETURN_GENERATED_KEYS);
-            statement.setString(1, StringUtils.left(action.getUserDn(), 255));
-            statement.setString(2, StringUtils.left(action.getUserName(), 100));
-            statement.setString(3, action.getSourceId());
-            statement.setString(4, StringUtils.left(action.getSourceName(), 1000));
-            statement.setString(5, action.getSourceType().toString());
-            statement.setString(6, action.getOperation().toString());
-            statement.setTimestamp(7, new java.sql.Timestamp(action.getTimestamp().getTime()));
-
-            // insert the action
-            int updateCount = statement.executeUpdate();
-
-            // get the action id
-            rs = statement.getGeneratedKeys();
-            if (updateCount == 1 && rs.next()) {
-                action.setId(rs.getInt(1));
-            } else {
-                throw new DataAccessException("Unable to insert action.");
-            }
-
-            // close the previous statement
-            statement.close();
-
-            // determine the type of component
-            ComponentDetails componentDetails = action.getComponentDetails();
-            if (componentDetails instanceof ProcessorDetails) {
-                createProcessorDetails(action.getId(), (ProcessorDetails) componentDetails);
-            } else if (componentDetails instanceof RemoteProcessGroupDetails) {
-                createRemoteProcessGroupDetails(action.getId(), (RemoteProcessGroupDetails) componentDetails);
-            }
-
-            // determine the type of action
-            ActionDetails details = action.getActionDetails();
-            if (details instanceof ConnectDetails) {
-                createConnectDetails(action.getId(), (ConnectDetails) details);
-            } else if (details instanceof MoveDetails) {
-                createMoveDetails(action.getId(), (MoveDetails) details);
-            } else if (details instanceof ConfigureDetails) {
-                createConfigureDetails(action.getId(), (ConfigureDetails) details);
-            } else if (details instanceof PurgeDetails) {
-                createPurgeDetails(action.getId(), (PurgeDetails) details);
-            }
-
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    /**
-     * Persists the processor details.
-     *
-     * @param actionId
-     * @param processorDetails
-     * @throws DataAccessException
-     */
-    private void createProcessorDetails(int actionId, ProcessorDetails processorDetails) throws DataAccessException {
-        PreparedStatement statement = null;
-        try {
-            // obtain a statement to insert to the processor action table
-            statement = connection.prepareStatement(INSERT_PROCESSOR_DETAILS);
-            statement.setInt(1, actionId);
-            statement.setString(2, StringUtils.left(processorDetails.getType(), 1000));
-
-            // insert the action
-            int updateCount = statement.executeUpdate();
-
-            // ensure the operation completed successfully
-            if (updateCount != 1) {
-                throw new DataAccessException("Unable to insert processor details.");
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    /**
-     * Persists the remote process group details.
-     *
-     * @param actionId
-     * @param remoteProcessGroupDetails
-     * @throws DataAccessException
-     */
-    private void createRemoteProcessGroupDetails(int actionId, RemoteProcessGroupDetails remoteProcessGroupDetails) throws DataAccessException {
-        PreparedStatement statement = null;
-        try {
-            // obtain a statement to insert to the processor action table
-            statement = connection.prepareStatement(INSERT_REMOTE_PROCESS_GROUP_DETAILS);
-            statement.setInt(1, actionId);
-            statement.setString(2, StringUtils.left(remoteProcessGroupDetails.getUri(), 2500));
-
-            // insert the action
-            int updateCount = statement.executeUpdate();
-
-            // ensure the operation completed successfully
-            if (updateCount != 1) {
-                throw new DataAccessException("Unable to insert remote prcoess group details.");
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    /**
-     * Persists the connection details.
-     *
-     * @param actionId
-     * @param connectionDetails
-     * @throws DataAccessException
-     */
-    private void createConnectDetails(int actionId, ConnectDetails connectionDetails) throws DataAccessException {
-        PreparedStatement statement = null;
-        try {
-            // obtain a statement to insert to the processor action table
-            statement = connection.prepareStatement(INSERT_CONNECT_DETAILS);
-            statement.setInt(1, actionId);
-            statement.setString(2, connectionDetails.getSourceId());
-            statement.setString(3, StringUtils.left(connectionDetails.getSourceName(), 1000));
-            statement.setString(4, StringUtils.left(connectionDetails.getSourceType().toString(), 1000));
-            statement.setString(5, StringUtils.left(connectionDetails.getRelationship(), 1000));
-            statement.setString(6, connectionDetails.getDestinationId());
-            statement.setString(7, StringUtils.left(connectionDetails.getDestinationName(), 1000));
-            statement.setString(8, StringUtils.left(connectionDetails.getDestinationType().toString(), 1000));
-
-            // insert the action
-            int updateCount = statement.executeUpdate();
-
-            // ensure the operation completed successfully
-            if (updateCount != 1) {
-                throw new DataAccessException("Unable to insert connection details.");
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    /**
-     * Persists the move details.
-     *
-     * @param actionId
-     * @param moveDetails
-     * @throws DataAccessException
-     */
-    private void createMoveDetails(int actionId, MoveDetails moveDetails) throws DataAccessException {
-        PreparedStatement statement = null;
-        try {
-            // obtain a statement to insert to the processor action table
-            statement = connection.prepareStatement(INSERT_MOVE_DETAILS);
-            statement.setInt(1, actionId);
-            statement.setString(2, moveDetails.getGroupId());
-            statement.setString(3, StringUtils.left(moveDetails.getGroup(), 1000));
-            statement.setString(4, moveDetails.getPreviousGroupId());
-            statement.setString(5, StringUtils.left(moveDetails.getPreviousGroup(), 1000));
-
-            // insert the action
-            int updateCount = statement.executeUpdate();
-
-            // ensure the operation completed successfully
-            if (updateCount != 1) {
-                throw new DataAccessException("Unable to insert move details.");
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    /**
-     * Persists the configuration details.
-     *
-     * @param actionId
-     * @param configurationDetails
-     * @throws DataAccessException
-     */
-    private void createConfigureDetails(int actionId, ConfigureDetails configurationDetails) throws DataAccessException {
-        PreparedStatement statement = null;
-        try {
-            // obtain a statement to insert to the processor action table
-            statement = connection.prepareStatement(INSERT_CONFIGURE_DETAILS);
-            statement.setInt(1, actionId);
-            statement.setString(2, StringUtils.left(configurationDetails.getName(), 1000));
-            statement.setString(3, StringUtils.left(configurationDetails.getValue(), 5000));
-            statement.setString(4, StringUtils.left(configurationDetails.getPreviousValue(), 5000));
-
-            // insert the action
-            int updateCount = statement.executeUpdate();
-
-            // ensure the operation completed successfully
-            if (updateCount != 1) {
-                throw new DataAccessException("Unable to insert configure details.");
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    /**
-     * Persists the purge details.
-     *
-     * @param actionId
-     * @param purgeDetails
-     * @throws DataAccessException
-     */
-    private void createPurgeDetails(int actionId, PurgeDetails purgeDetails) throws DataAccessException {
-        PreparedStatement statement = null;
-        try {
-            // obtain a statement to insert to the processor action table
-            statement = connection.prepareStatement(INSERT_PURGE_DETAILS);
-            statement.setInt(1, actionId);
-            statement.setTimestamp(2, new java.sql.Timestamp(purgeDetails.getEndDate().getTime()));
-
-            // insert the action
-            int updateCount = statement.executeUpdate();
-
-            // ensure the operation completed successfully
-            if (updateCount != 1) {
-                throw new DataAccessException("Unable to insert connection details.");
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    /**
-     * Finds actions that meet the criteria in the specified query.
-     *
-     * @param historyQuery
-     * @return
-     * @throws DataAccessException
-     */
-    @Override
-    public History findActions(HistoryQuery historyQuery) throws DataAccessException {
-
-        // get the sort column
-        String sortColumn = "ACTION_TIMESTAMP";
-        if (StringUtils.isNotBlank(historyQuery.getSortColumn())) {
-            String rawColumnName = historyQuery.getSortColumn();
-            if (!columnMap.containsKey(rawColumnName)) {
-                throw new IllegalArgumentException(String.format("Unrecognized column name '%s'.", rawColumnName));
-            }
-            sortColumn = columnMap.get(rawColumnName);
-        }
-
-        // get the sort order
-        String sortOrder = "desc";
-        if (StringUtils.isNotBlank(historyQuery.getSortOrder())) {
-            sortOrder = historyQuery.getSortOrder();
-        }
-
-        History actionResult = new History();
-        Collection<Action> actions = new ArrayList<>();
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            List<String> where = new ArrayList<>();
-
-            // append the start time
-            if (historyQuery.getStartDate() != null) {
-                where.add("ACTION_TIMESTAMP >= ?");
-            }
-
-            // append the end time
-            if (historyQuery.getEndDate() != null) {
-                where.add("ACTION_TIMESTAMP <= ?");
-            }
-
-            // append the user id as necessary
-            if (historyQuery.getUserName() != null) {
-                where.add("UPPER(USER_NAME) LIKE ?");
-            }
-
-            // append the source id as necessary
-            if (historyQuery.getSourceId() != null) {
-                where.add("SOURCE_ID = ?");
-            }
-
-            String sql = SELECT_ACTION_COUNT;
-            if (!where.isEmpty()) {
-                sql += " WHERE " + StringUtils.join(where, " AND ");
-            }
-
-            // get the total number of actions
-            statement = connection.prepareStatement(sql);
-            int paramIndex = 1;
-
-            // set the start date as necessary
-            if (historyQuery.getStartDate() != null) {
-                statement.setTimestamp(paramIndex++, new java.sql.Timestamp(historyQuery.getStartDate().getTime()));
-            }
-
-            // set the end date as necessary
-            if (historyQuery.getEndDate() != null) {
-                statement.setTimestamp(paramIndex++, new java.sql.Timestamp(historyQuery.getEndDate().getTime()));
-            }
-
-            // set the user id as necessary
-            if (historyQuery.getUserName() != null) {
-                statement.setString(paramIndex++, "%" + historyQuery.getUserName().toUpperCase() + "%");
-            }
-
-            // set the source id as necessary
-            if (historyQuery.getSourceId() != null) {
-                statement.setString(paramIndex, historyQuery.getSourceId());
-            }
-
-            // execute the statement
-            rs = statement.executeQuery();
-
-            // ensure there are results
-            if (rs.next()) {
-                actionResult.setTotal(rs.getInt("ACTION_COUNT"));
-            } else {
-                throw new DataAccessException("Unable to determine total action count.");
-            }
-
-            sql = SELECT_ACTIONS;
-            if (!where.isEmpty()) {
-                sql += " WHERE " + StringUtils.join(where, " AND ");
-            }
-
-            // append the sort criteria
-            sql += (" ORDER BY " + sortColumn + " " + sortOrder);
-
-            // append the offset and limit
-            sql += " LIMIT ? OFFSET ?";
-
-            // close the previous statement
-            statement.close();
-
-            // create the statement
-            statement = connection.prepareStatement(sql);
-            paramIndex = 1;
-
-            // set the start date as necessary
-            if (historyQuery.getStartDate() != null) {
-                statement.setTimestamp(paramIndex++, new java.sql.Timestamp(historyQuery.getStartDate().getTime()));
-            }
-
-            // set the end date as necessary
-            if (historyQuery.getEndDate() != null) {
-                statement.setTimestamp(paramIndex++, new java.sql.Timestamp(historyQuery.getEndDate().getTime()));
-            }
-
-            // set the user id as necessary
-            if (historyQuery.getUserName() != null) {
-                statement.setString(paramIndex++, "%" + historyQuery.getUserName().toUpperCase() + "%");
-            }
-
-            // set the source id as necessary
-            if (historyQuery.getSourceId() != null) {
-                statement.setString(paramIndex++, historyQuery.getSourceId());
-            }
-
-            // set the limit
-            statement.setInt(paramIndex++, historyQuery.getCount());
-
-            // set the offset according to the currented page calculated above
-            statement.setInt(paramIndex, historyQuery.getOffset());
-
-            // execute the query
-            rs = statement.executeQuery();
-
-            // create each corresponding action
-            while (rs.next()) {
-                final Integer actionId = rs.getInt("ID");
-                final Operation operation = Operation.valueOf(rs.getString("OPERATION"));
-                final Component component = Component.valueOf(rs.getString("SOURCE_TYPE"));
-
-                Action action = new Action();
-                action.setId(actionId);
-                action.setUserDn(rs.getString("USER_DN"));
-                action.setUserName(rs.getString("USER_NAME"));
-                action.setOperation(Operation.valueOf(rs.getString("OPERATION")));
-                action.setTimestamp(new Date(rs.getTimestamp("ACTION_TIMESTAMP").getTime()));
-                action.setSourceId(rs.getString("SOURCE_ID"));
-                action.setSourceName(rs.getString("SOURCE_NAME"));
-                action.setSourceType(Component.valueOf(rs.getString("SOURCE_TYPE")));
-
-                // get the component details if appropriate
-                ComponentDetails componentDetails = null;
-                if (Component.Processor.equals(component)) {
-                    componentDetails = getProcessorDetails(actionId);
-                } else if (Component.RemoteProcessGroup.equals(component)) {
-                    componentDetails = getRemoteProcessGroupDetails(actionId);
-                }
-
-                if (componentDetails != null) {
-                    action.setComponentDetails(componentDetails);
-                }
-
-                // get the action details if appropriate
-                ActionDetails actionDetails = null;
-                if (Operation.Move.equals(operation)) {
-                    actionDetails = getMoveDetails(actionId);
-                } else if (Operation.Configure.equals(operation)) {
-                    actionDetails = getConfigureDetails(actionId);
-                } else if (Operation.Connect.equals(operation) || Operation.Disconnect.equals(operation)) {
-                    actionDetails = getConnectDetails(actionId);
-                } else if (Operation.Purge.equals(operation)) {
-                    actionDetails = getPurgeDetails(actionId);
-                }
-
-                // set the action details
-                if (actionDetails != null) {
-                    action.setActionDetails(actionDetails);
-                }
-
-                // add the action
-                actions.add(action);
-            }
-
-            // populate the action result
-            actionResult.setActions(actions);
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-
-        return actionResult;
-    }
-
-    @Override
-    public Action getAction(Integer actionId) throws DataAccessException {
-        Action action = null;
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            // create the statement
-            statement = connection.prepareStatement(SELECT_ACTION_BY_ID);
-            statement.setInt(1, actionId);
-
-            // execute the query
-            rs = statement.executeQuery();
-
-            // ensure results
-            if (rs.next()) {
-                Operation operation = Operation.valueOf(rs.getString("OPERATION"));
-                Component component = Component.valueOf(rs.getString("SOURCE_TYPE"));
-
-                // populate the action
-                action = new Action();
-                action.setId(rs.getInt("ID"));
-                action.setUserDn(rs.getString("USER_DN"));
-                action.setUserName(rs.getString("USER_NAME"));
-                action.setOperation(operation);
-                action.setTimestamp(new Date(rs.getTimestamp("ACTION_TIMESTAMP").getTime()));
-                action.setSourceId(rs.getString("SOURCE_ID"));
-                action.setSourceName(rs.getString("SOURCE_NAME"));
-                action.setSourceType(component);
-
-                // get the component details if appropriate
-                ComponentDetails componentDetails = null;
-                if (Component.Processor.equals(component)) {
-                    componentDetails = getProcessorDetails(actionId);
-                } else if (Component.RemoteProcessGroup.equals(component)) {
-                    componentDetails = getRemoteProcessGroupDetails(actionId);
-                }
-
-                if (componentDetails != null) {
-                    action.setComponentDetails(componentDetails);
-                }
-
-                // get the action details if appropriate
-                ActionDetails actionDetails = null;
-                if (Operation.Move.equals(operation)) {
-                    actionDetails = getMoveDetails(actionId);
-                } else if (Operation.Configure.equals(operation)) {
-                    actionDetails = getConfigureDetails(actionId);
-                } else if (Operation.Connect.equals(operation) || Operation.Disconnect.equals(operation)) {
-                    actionDetails = getConnectDetails(actionId);
-                } else if (Operation.Purge.equals(operation)) {
-                    actionDetails = getPurgeDetails(actionId);
-                }
-
-                // set the action details
-                if (actionDetails != null) {
-                    action.setActionDetails(actionDetails);
-                }
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-
-        return action;
-    }
-
-    /**
-     * Loads the specified processor details.
-     *
-     * @param actionId
-     * @return
-     * @throws DataAccessException
-     */
-    private ProcessorDetails getProcessorDetails(Integer actionId) throws DataAccessException {
-        ProcessorDetails processorDetails = null;
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            // create the statement
-            statement = connection.prepareStatement(SELECT_PROCESSOR_DETAILS_FOR_ACTION);
-            statement.setInt(1, actionId);
-
-            // execute the query
-            rs = statement.executeQuery();
-
-            // ensure results
-            if (rs.next()) {
-                processorDetails = new ProcessorDetails();
-                processorDetails.setType(rs.getString("TYPE"));
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-
-        return processorDetails;
-    }
-
-    /**
-     * Loads the specified remote process group details.
-     *
-     * @param actionId
-     * @return
-     * @throws DataAccessException
-     */
-    private RemoteProcessGroupDetails getRemoteProcessGroupDetails(Integer actionId) throws DataAccessException {
-        RemoteProcessGroupDetails remoteProcessGroupDetails = null;
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            // create the statement
-            statement = connection.prepareStatement(SELECT_REMOTE_PROCESS_GROUP_DETAILS_FOR_ACTION);
-            statement.setInt(1, actionId);
-
-            // execute the query
-            rs = statement.executeQuery();
-
-            // ensure results
-            if (rs.next()) {
-                remoteProcessGroupDetails = new RemoteProcessGroupDetails();
-                remoteProcessGroupDetails.setUri(rs.getString("URI"));
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-
-        return remoteProcessGroupDetails;
-    }
-
-    /**
-     * Loads the specified move details.
-     *
-     * @param actionId
-     * @return
-     * @throws DataAccessException
-     */
-    private MoveDetails getMoveDetails(Integer actionId) throws DataAccessException {
-        MoveDetails moveDetails = null;
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            // create the statement
-            statement = connection.prepareStatement(SELECT_MOVE_DETAILS_FOR_ACTION);
-            statement.setInt(1, actionId);
-
-            // execute the query
-            rs = statement.executeQuery();
-
-            // ensure results
-            if (rs.next()) {
-                moveDetails = new MoveDetails();
-                moveDetails.setGroupId(rs.getString("GROUP_ID"));
-                moveDetails.setGroup(rs.getString("GROUP_NAME"));
-                moveDetails.setPreviousGroupId(rs.getString("PREVIOUS_GROUP_ID"));
-                moveDetails.setPreviousGroup(rs.getString("PREVIOUS_GROUP_NAME"));
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-
-        return moveDetails;
-    }
-
-    /**
-     * Loads the specified relationship details.
-     *
-     * @param actionId
-     * @return
-     * @throws DataAccessException
-     */
-    private ConnectDetails getConnectDetails(Integer actionId) throws DataAccessException {
-        ConnectDetails connectionDetails = null;
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            // create the statement
-            statement = connection.prepareStatement(SELECT_CONNECT_DETAILS_FOR_ACTION);
-            statement.setInt(1, actionId);
-
-            // execute the query
-            rs = statement.executeQuery();
-
-            // ensure results
-            if (rs.next()) {
-                final Component sourceComponent = Component.valueOf(rs.getString("SOURCE_TYPE"));
-                final Component destinationComponent = Component.valueOf(rs.getString("DESTINATION_TYPE"));
-
-                connectionDetails = new ConnectDetails();
-                connectionDetails.setSourceId(rs.getString("SOURCE_ID"));
-                connectionDetails.setSourceName(rs.getString("SOURCE_NAME"));
-                connectionDetails.setSourceType(sourceComponent);
-                connectionDetails.setRelationship(rs.getString("RELATIONSHIP"));
-                connectionDetails.setDestinationId(rs.getString("DESTINATION_ID"));
-                connectionDetails.setDestinationName(rs.getString("DESTINATION_NAME"));
-                connectionDetails.setDestinationType(destinationComponent);
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-
-        return connectionDetails;
-    }
-
-    /**
-     * Loads the specified configuration details.
-     *
-     * @param actionId
-     * @return
-     * @throws DataAccessException
-     */
-    private ConfigureDetails getConfigureDetails(Integer actionId) throws DataAccessException {
-        ConfigureDetails configurationDetails = null;
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            // create the statement
-            statement = connection.prepareStatement(SELECT_CONFIGURE_DETAILS_FOR_ACTION);
-            statement.setInt(1, actionId);
-
-            // execute the query
-            rs = statement.executeQuery();
-
-            // ensure results
-            if (rs.next()) {
-                configurationDetails = new ConfigureDetails();
-                configurationDetails.setName(rs.getString("NAME"));
-                configurationDetails.setValue(rs.getString("VALUE"));
-                configurationDetails.setPreviousValue(rs.getString("PREVIOUS_VALUE"));
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-
-        return configurationDetails;
-    }
-
-    /**
-     * Loads the specified purge details.
-     *
-     * @param actionId
-     * @return
-     * @throws DataAccessException
-     */
-    private PurgeDetails getPurgeDetails(Integer actionId) throws DataAccessException {
-        PurgeDetails purgeDetails = null;
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            // create the statement
-            statement = connection.prepareStatement(SELECT_PURGE_DETAILS_FOR_ACTION);
-            statement.setInt(1, actionId);
-
-            // execute the query
-            rs = statement.executeQuery();
-
-            // ensure results
-            if (rs.next()) {
-                purgeDetails = new PurgeDetails();
-                purgeDetails.setEndDate(new Date(rs.getTimestamp("END_DATE").getTime()));
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-
-        return purgeDetails;
-    }
-
-    @Override
-    public Map<String, List<PreviousValue>> getPreviousValues(String processorId) {
-        Map<String, List<PreviousValue>> previousValues = new LinkedHashMap<>();
-
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            // create the statement
-            statement = connection.prepareStatement(SELECT_PREVIOUSLY_CONFIGURED_FIELDS);
-            statement.setString(1, processorId);
-
-            // execute the query
-            rs = statement.executeQuery();
-
-            // ensure results
-            while (rs.next()) {
-                final String property = rs.getString("NAME");
-                previousValues.put(property, getPreviousValuesForProperty(processorId, property));
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-
-        return previousValues;
-    }
-
-    private List<PreviousValue> getPreviousValuesForProperty(final String processorId, final String property) {
-        List<PreviousValue> previousValues = new ArrayList<>();
-
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            // create the statement
-            statement = connection.prepareStatement(SELECT_PREVIOUS_VALUES);
-            statement.setString(1, processorId);
-            statement.setString(2, property);
-
-            // execute the query
-            rs = statement.executeQuery();
-
-            // ensure results
-            while (rs.next()) {
-                // get the previous value
-                final PreviousValue previousValue = new PreviousValue();
-                previousValue.setPreviousValue(rs.getString("VALUE"));
-                previousValue.setTimestamp(new Date(rs.getTimestamp("ACTION_TIMESTAMP").getTime()));
-                previousValue.setUserName(rs.getString("USER_NAME"));
-                previousValues.add(previousValue);
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-
-        return previousValues;
-    }
-
-    @Override
-    public void deleteActions(Date endDate) throws DataAccessException {
-        PreparedStatement statement = null;
-        try {
-            // -----------------
-            // component details
-            // -----------------
-
-            // create the move delete statement
-            statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "PROCESSOR_DETAILS", "ACTION_ID"));
-            statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime()));
-            statement.executeUpdate();
-            statement.close();
-
-            // create the move delete statement
-            statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "REMOTE_PROCESS_GROUP_DETAILS", "ACTION_ID"));
-            statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime()));
-            statement.executeUpdate();
-            statement.close();
-
-            // --------------
-            // action details
-            // --------------
-            // create the move delete statement
-            statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "MOVE_DETAILS", "ACTION_ID"));
-            statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime()));
-            statement.executeUpdate();
-            statement.close();
-
-            // create the configure delete statement
-            statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "CONFIGURE_DETAILS", "ACTION_ID"));
-            statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime()));
-            statement.executeUpdate();
-            statement.close();
-
-            // create the connect delete statement
-            statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "CONNECT_DETAILS", "ACTION_ID"));
-            statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime()));
-            statement.executeUpdate();
-            statement.close();
-
-            // create the relationship delete statement
-            statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "PURGE_DETAILS", "ACTION_ID"));
-            statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime()));
-            statement.executeUpdate();
-            statement.close();
-
-            // -------
-            // actions
-            // -------
-            // create the action delete statement
-            statement = connection.prepareStatement(DELETE_ACTIONS);
-            statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime()));
-            statement.executeUpdate();
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardAuthorityDAO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardAuthorityDAO.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardAuthorityDAO.java
deleted file mode 100644
index 4e2cc26..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardAuthorityDAO.java
+++ /dev/null
@@ -1,172 +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.admin.dao.impl;
-
-import java.sql.Connection;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.util.EnumSet;
-import java.util.Set;
-import org.apache.nifi.admin.RepositoryUtils;
-import org.apache.nifi.admin.dao.AuthorityDAO;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.authorization.Authority;
-
-/**
- *
- */
-public class StandardAuthorityDAO implements AuthorityDAO {
-
-    private static final String SELECT_AUTHORITIES_FOR_USER = "SELECT ID, ROLE "
-            + "FROM AUTHORITY "
-            + "WHERE USER_ID = ?";
-
-    private static final String INSERT_AUTHORITY = "INSERT INTO AUTHORITY ("
-            + "USER_ID, ROLE"
-            + ") VALUES ("
-            + "?, ?"
-            + ")";
-
-    private static final String DELETE_AUTHORITY = "DELETE FROM AUTHORITY "
-            + "WHERE USER_ID = ? AND ROLE = ?";
-
-    private static final String DELETE_AUTHORITIES_FOR_USER = "DELETE FROM AUTHORITY "
-            + "WHERE USER_ID = ?";
-
-    private final Connection connection;
-
-    public StandardAuthorityDAO(Connection connection) {
-        this.connection = connection;
-    }
-
-    @Override
-    public void createAuthorities(Set<Authority> authorities, String userId) throws DataAccessException {
-        if (authorities == null) {
-            throw new IllegalArgumentException("Specified authorities cannot be null.");
-        }
-
-        // ensure there are some authorities to create
-        if (!authorities.isEmpty()) {
-            PreparedStatement statement = null;
-            try {
-                // add each authority for the specified user
-                statement = connection.prepareStatement(INSERT_AUTHORITY);
-                statement.setString(1, userId);
-                for (Authority authority : authorities) {
-                    statement.setString(2, authority.toString());
-                    statement.addBatch();
-                }
-
-                // insert the authorities
-                int[] updateCounts = statement.executeBatch();
-                for (int updateCount : updateCounts) {
-                    if (updateCount != 1) {
-                        throw new DataAccessException("Unable to insert user authorities.");
-                    }
-                }
-            } catch (SQLException sqle) {
-                throw new DataAccessException(sqle);
-            } catch (DataAccessException dae) {
-                throw dae;
-            } finally {
-                RepositoryUtils.closeQuietly(statement);
-            }
-        }
-    }
-
-    @Override
-    public void deleteAuthorities(String userId) throws DataAccessException {
-        // ensure there are some authorities to create
-        PreparedStatement statement = null;
-        try {
-            // add each authority for the specified user
-            statement = connection.prepareStatement(DELETE_AUTHORITIES_FOR_USER);
-            statement.setString(1, userId);
-
-            // insert the authorities
-            statement.executeUpdate();
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    @Override
-    public void deleteAuthorities(Set<Authority> authorities, String userId) throws DataAccessException {
-        if (authorities == null) {
-            throw new IllegalArgumentException("Specified authorities cannot be null.");
-        }
-
-        // ensure there are some authorities to create
-        if (!authorities.isEmpty()) {
-            PreparedStatement statement = null;
-            try {
-                // add each authority for the specified user
-                statement = connection.prepareStatement(DELETE_AUTHORITY);
-                statement.setString(1, userId);
-                for (Authority authority : authorities) {
-                    statement.setString(2, authority.toString());
-                    statement.addBatch();
-                }
-
-                // insert the authorities
-                int[] updateCounts = statement.executeBatch();
-                for (int updateCount : updateCounts) {
-                    if (updateCount != 1) {
-                        throw new DataAccessException("Unable to remove user authorities.");
-                    }
-                }
-            } catch (SQLException sqle) {
-                throw new DataAccessException(sqle);
-            } catch (DataAccessException dae) {
-                throw dae;
-            } finally {
-                RepositoryUtils.closeQuietly(statement);
-            }
-        }
-    }
-
-    @Override
-    public Set<Authority> findAuthoritiesByUserId(String userId) throws DataAccessException {
-        Set<Authority> authorities = EnumSet.noneOf(Authority.class);
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            // add each authority for the specified user
-            statement = connection.prepareStatement(SELECT_AUTHORITIES_FOR_USER);
-            statement.setString(1, userId);
-
-            // execute the query
-            rs = statement.executeQuery();
-
-            // create each corresponding authority
-            while (rs.next()) {
-                authorities.add(Authority.valueOfAuthority(rs.getString("ROLE")));
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-
-        return authorities;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java
deleted file mode 100644
index ea7c1a1..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java
+++ /dev/null
@@ -1,634 +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.admin.dao.impl;
-
-import java.nio.charset.StandardCharsets;
-import java.sql.Connection;
-import org.apache.nifi.admin.dao.UserDAO;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.sql.Types;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.UUID;
-import org.apache.nifi.admin.RepositoryUtils;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.authorization.Authority;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUser;
-
-/**
- * Responsible for loading and persisting NiFiUsers.
- */
-public class StandardUserDAO implements UserDAO {
-
-    private static final String SELECT_PENDING_ACCOUNTS_COUNT = "SELECT "
-            + "COUNT(*) as PENDING_ACCOUNTS "
-            + "FROM USER U "
-            + "WHERE U.STATUS = 'PENDING'";
-
-    private static final String SELECT_USER_BY_DN = "SELECT "
-            + "U.ID, "
-            + "U.DN, "
-            + "U.USER_NAME, "
-            + "U.USER_GROUP, "
-            + "U.CREATION, "
-            + "U.LAST_ACCESSED, "
-            + "U.LAST_VERIFIED, "
-            + "U.JUSTIFICATION, "
-            + "U.STATUS, "
-            + "A.ROLE "
-            + "FROM USER U "
-            + "LEFT JOIN AUTHORITY A " // ensures that users without authorities are still matched
-            + "ON U.ID = A.USER_ID "
-            + "WHERE U.DN = ?";
-
-    private static final String SELECT_USER_BY_ID = "SELECT "
-            + "U.ID, "
-            + "U.DN, "
-            + "U.USER_NAME, "
-            + "U.USER_GROUP, "
-            + "U.CREATION, "
-            + "U.LAST_ACCESSED, "
-            + "U.LAST_VERIFIED, "
-            + "U.JUSTIFICATION, "
-            + "U.STATUS, "
-            + "A.ROLE "
-            + "FROM USER U "
-            + "LEFT JOIN AUTHORITY A " // ensures that users without authorities are still matched
-            + "ON U.ID = A.USER_ID "
-            + "WHERE U.ID = ?";
-
-    private static final String SELECT_USERS = "SELECT "
-            + "U.ID, "
-            + "U.DN, "
-            + "U.USER_NAME, "
-            + "U.USER_GROUP, "
-            + "U.CREATION, "
-            + "U.LAST_ACCESSED, "
-            + "U.LAST_VERIFIED, "
-            + "U.JUSTIFICATION, "
-            + "U.STATUS, "
-            + "A.ROLE "
-            + "FROM USER U "
-            + "LEFT JOIN AUTHORITY A " // ensures that users without authorities are still matched
-            + "ON U.ID = A.USER_ID "
-            + "WHERE U.DN <> ?";
-
-    private static final String SELECT_USER_GROUPS = "SELECT DISTINCT "
-            + "U.USER_GROUP "
-            + "FROM USER U";
-
-    private static final String SELECT_USER_GROUP = "SELECT "
-            + "U.ID, "
-            + "U.DN, "
-            + "U.USER_NAME, "
-            + "U.USER_GROUP, "
-            + "U.CREATION, "
-            + "U.LAST_ACCESSED, "
-            + "U.LAST_VERIFIED, "
-            + "U.JUSTIFICATION, "
-            + "U.STATUS, "
-            + "A.ROLE "
-            + "FROM USER U "
-            + "LEFT JOIN AUTHORITY A " // ensures that users without authorities are still matched
-            + "ON U.ID = A.USER_ID "
-            + "WHERE U.DN <> ? AND U.USER_GROUP = ?";
-
-    private static final String INSERT_USER = "INSERT INTO USER ("
-            + "ID, DN, USER_NAME, USER_GROUP, CREATION, LAST_VERIFIED, JUSTIFICATION, STATUS"
-            + ") VALUES ("
-            + "?, "
-            + "?, "
-            + "?, "
-            + "?, "
-            + "NOW(), "
-            + "?, "
-            + "?, "
-            + "?"
-            + ")";
-
-    private static final String UPDATE_USER = "UPDATE USER SET "
-            + "DN = ?, "
-            + "USER_NAME = ?, "
-            + "USER_GROUP = ?, "
-            + "LAST_ACCESSED = ?, "
-            + "LAST_VERIFIED = ?, "
-            + "JUSTIFICATION = ?, "
-            + "STATUS = ? "
-            + "WHERE ID = ?";
-
-    private static final String UPDATE_USER_GROUP_STATUS = "UPDATE USER SET "
-            + "STATUS = ?,"
-            + "USER_GROUP = NULL "
-            + "WHERE USER_GROUP = ?";
-
-    private static final String UPDATE_USER_GROUP_VERIFICATION = "UPDATE USER SET "
-            + "LAST_VERIFIED = ? "
-            + "WHERE USER_GROUP = ?";
-
-    private static final String UNGROUP_GROUP = "UPDATE USER SET "
-            + "USER_GROUP = NULL "
-            + "WHERE USER_GROUP = ?";
-
-    private static final String DELETE_USER = "DELETE FROM USER "
-            + "WHERE ID = ?";
-
-    private final Connection connection;
-
-    public StandardUserDAO(Connection connection) {
-        this.connection = connection;
-    }
-
-    @Override
-    public Boolean hasPendingUserAccounts() throws DataAccessException {
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            // create the connection and obtain a statement
-            statement = connection.prepareStatement(SELECT_PENDING_ACCOUNTS_COUNT);
-
-            // execute the query
-            rs = statement.executeQuery();
-
-            // get the first row which will contain the number of pending accounts
-            if (rs.next()) {
-                int pendingAccounts = rs.getInt("PENDING_ACCOUNTS");
-                return pendingAccounts > 0;
-            }
-
-            // query returned no results?
-            return false;
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    @Override
-    public Set<NiFiUser> findUsers() throws DataAccessException {
-        Set<NiFiUser> users = new HashSet<>();
-
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            // create the connection and obtain a statement
-            statement = connection.prepareStatement(SELECT_USERS);
-            statement.setString(1, NiFiUser.ANONYMOUS_USER_DN);
-
-            // execute the query
-            rs = statement.executeQuery();
-
-            // create the user
-            NiFiUser user = null;
-
-            // go through the user and its roles
-            while (rs.next()) {
-                // get the user id for the current record
-                String userId = rs.getString("ID");
-
-                // create the user during the first iteration
-                if (user == null || !userId.equals(user.getId())) {
-                    user = new NiFiUser();
-                    user.setId(userId);
-                    user.setDn(rs.getString("DN"));
-                    user.setUserName(rs.getString("USER_NAME"));
-                    user.setUserGroup(rs.getString("USER_GROUP"));
-                    user.setJustification(rs.getString("JUSTIFICATION"));
-                    user.setStatus(AccountStatus.valueOfStatus(rs.getString("STATUS")));
-
-                    // set the creation date
-                    user.setCreation(new Date(rs.getTimestamp("CREATION").getTime()));
-
-                    // get the last accessed date
-                    if (rs.getTimestamp("LAST_ACCESSED") != null) {
-                        user.setLastAccessed(new Date(rs.getTimestamp("LAST_ACCESSED").getTime()));
-                    }
-
-                    // get the last verified date
-                    if (rs.getTimestamp("LAST_VERIFIED") != null) {
-                        user.setLastVerified(new Date(rs.getTimestamp("LAST_VERIFIED").getTime()));
-                    }
-
-                    // add the user
-                    users.add(user);
-                }
-
-                // the select statement performs a left join since the desired
-                // user may not have any authorities
-                String authority = rs.getString("ROLE");
-                if (StringUtils.isNotBlank(authority)) {
-                    user.getAuthorities().add(Authority.valueOfAuthority(authority));
-                }
-            }
-
-            return users;
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    @Override
-    public Set<String> findUserGroups() throws DataAccessException {
-        Set<String> userGroups = new HashSet<>();
-
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            // create the connection and obtain a statement
-            statement = connection.prepareStatement(SELECT_USER_GROUPS);
-
-            // execute the query
-            rs = statement.executeQuery();
-
-            // get each user group
-            while (rs.next()) {
-                userGroups.add(rs.getString("USER_GROUP"));
-            }
-
-            return userGroups;
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    @Override
-    public Set<NiFiUser> findUsersForGroup(String group) throws DataAccessException {
-        Set<NiFiUser> users = new HashSet<>();
-
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            // create the connection and obtain a statement
-            statement = connection.prepareStatement(SELECT_USER_GROUP);
-            statement.setString(1, NiFiUser.ANONYMOUS_USER_DN);
-            statement.setString(2, group);
-
-            // execute the query
-            rs = statement.executeQuery();
-
-            // create the user
-            NiFiUser user = null;
-
-            // go through the user and its roles
-            while (rs.next()) {
-                // get the user id for the current record
-                String userId = rs.getString("ID");
-
-                // create the user during the first iteration
-                if (user == null || !userId.equals(user.getId())) {
-                    user = new NiFiUser();
-                    user.setId(userId);
-                    user.setDn(rs.getString("DN"));
-                    user.setUserName(rs.getString("USER_NAME"));
-                    user.setUserGroup(rs.getString("USER_GROUP"));
-                    user.setJustification(rs.getString("JUSTIFICATION"));
-                    user.setStatus(AccountStatus.valueOfStatus(rs.getString("STATUS")));
-
-                    // set the creation date
-                    user.setCreation(new Date(rs.getTimestamp("CREATION").getTime()));
-
-                    // get the last accessed date
-                    if (rs.getTimestamp("LAST_ACCESSED") != null) {
-                        user.setLastAccessed(new Date(rs.getTimestamp("LAST_ACCESSED").getTime()));
-                    }
-
-                    // get the last verified date
-                    if (rs.getTimestamp("LAST_VERIFIED") != null) {
-                        user.setLastVerified(new Date(rs.getTimestamp("LAST_VERIFIED").getTime()));
-                    }
-
-                    // add the user
-                    users.add(user);
-                }
-
-                // the select statement performs a left join since the desired
-                // user may not have any authorities
-                String authority = rs.getString("ROLE");
-                if (StringUtils.isNotBlank(authority)) {
-                    user.getAuthorities().add(Authority.valueOfAuthority(authority));
-                }
-            }
-
-            return users;
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    @Override
-    public NiFiUser findUserById(String id) throws DataAccessException {
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            // create the connection and obtain a statement
-            statement = connection.prepareStatement(SELECT_USER_BY_ID);
-            statement.setString(1, id);
-
-            // execute the query
-            rs = statement.executeQuery();
-
-            // create the user
-            NiFiUser user = null;
-
-            // go through the user and its roles
-            while (rs.next()) {
-                // create the user during the first iteration
-                if (user == null) {
-                    user = new NiFiUser();
-                    user.setId(rs.getString("ID"));
-                    user.setDn(rs.getString("DN"));
-                    user.setUserName(rs.getString("USER_NAME"));
-                    user.setUserGroup(rs.getString("USER_GROUP"));
-                    user.setJustification(rs.getString("JUSTIFICATION"));
-                    user.setStatus(AccountStatus.valueOfStatus(rs.getString("STATUS")));
-
-                    // set the creation date
-                    user.setCreation(new Date(rs.getTimestamp("CREATION").getTime()));
-
-                    // get the last accessed date
-                    if (rs.getTimestamp("LAST_ACCESSED") != null) {
-                        user.setLastAccessed(new Date(rs.getTimestamp("LAST_ACCESSED").getTime()));
-                    }
-
-                    // get the last verified date
-                    if (rs.getTimestamp("LAST_VERIFIED") != null) {
-                        user.setLastVerified(new Date(rs.getTimestamp("LAST_VERIFIED").getTime()));
-                    }
-                }
-
-                // the select statement performs a left join since the desired
-                // user may not have any authorities
-                String authority = rs.getString("ROLE");
-                if (StringUtils.isNotBlank(authority)) {
-                    user.getAuthorities().add(Authority.valueOfAuthority(authority));
-                }
-            }
-
-            return user;
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    @Override
-    public NiFiUser findUserByDn(String dn) throws DataAccessException {
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            // create the connection and obtain a statement
-            statement = connection.prepareStatement(SELECT_USER_BY_DN);
-            statement.setString(1, dn);
-
-            // execute the query
-            rs = statement.executeQuery();
-
-            // create the user
-            NiFiUser user = null;
-
-            // go through the user and its roles
-            while (rs.next()) {
-                // create the user during the first iteration
-                if (user == null) {
-                    user = new NiFiUser();
-                    user.setId(rs.getString("ID"));
-                    user.setDn(rs.getString("DN"));
-                    user.setUserName(rs.getString("USER_NAME"));
-                    user.setUserGroup(rs.getString("USER_GROUP"));
-                    user.setJustification(rs.getString("JUSTIFICATION"));
-                    user.setStatus(AccountStatus.valueOfStatus(rs.getString("STATUS")));
-
-                    // set the creation date
-                    user.setCreation(new Date(rs.getTimestamp("CREATION").getTime()));
-
-                    // get the last accessed date
-                    if (rs.getTimestamp("LAST_ACCESSED") != null) {
-                        user.setLastAccessed(new Date(rs.getTimestamp("LAST_ACCESSED").getTime()));
-                    }
-
-                    // get the last verified date
-                    if (rs.getTimestamp("LAST_VERIFIED") != null) {
-                        user.setLastVerified(new Date(rs.getTimestamp("LAST_VERIFIED").getTime()));
-                    }
-                }
-
-                // the select statement performs a left join since the desired
-                // user may not have any authorities
-                String authority = rs.getString("ROLE");
-                if (StringUtils.isNotBlank(authority)) {
-                    user.getAuthorities().add(Authority.valueOfAuthority(authority));
-                }
-            }
-
-            return user;
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    @Override
-    public void createUser(NiFiUser user) throws DataAccessException {
-        if (user.getDn() == null) {
-            throw new IllegalArgumentException("User dn must be specified.");
-        }
-
-        PreparedStatement statement = null;
-        ResultSet rs = null;
-        try {
-            final String id = UUID.nameUUIDFromBytes(user.getDn().getBytes(StandardCharsets.UTF_8)).toString();
-
-            // create a statement
-            statement = connection.prepareStatement(INSERT_USER, Statement.RETURN_GENERATED_KEYS);
-            statement.setString(1, id);
-            statement.setString(2, StringUtils.left(user.getDn(), 255));
-            statement.setString(3, StringUtils.left(user.getUserName(), 100));
-            statement.setString(4, StringUtils.left(user.getUserGroup(), 100));
-            if (user.getLastVerified() != null) {
-                statement.setTimestamp(5, new java.sql.Timestamp(user.getLastVerified().getTime()));
-            } else {
-                statement.setTimestamp(5, null);
-            }
-            statement.setString(6, StringUtils.left(user.getJustification(), 500));
-            statement.setString(7, user.getStatus().toString());
-
-            // insert the user
-            int updateCount = statement.executeUpdate();
-            if (updateCount == 1) {
-                user.setId(id);
-            } else {
-                throw new DataAccessException("Unable to insert user.");
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } catch (DataAccessException dae) {
-            throw dae;
-        } finally {
-            RepositoryUtils.closeQuietly(rs);
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    @Override
-    public void deleteUser(String id) throws DataAccessException {
-        // ensure there are some authorities to create
-        PreparedStatement statement = null;
-        try {
-            // add each authority for the specified user
-            statement = connection.prepareStatement(DELETE_USER);
-            statement.setString(1, id);
-
-            // insert the authorities
-            statement.executeUpdate();
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } catch (DataAccessException dae) {
-            throw dae;
-        } finally {
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    @Override
-    public void updateUser(NiFiUser user) throws DataAccessException {
-        PreparedStatement statement = null;
-        try {
-            // create a statement
-            statement = connection.prepareStatement(UPDATE_USER);
-            statement.setString(1, StringUtils.left(user.getDn(), 255));
-            statement.setString(2, StringUtils.left(user.getUserName(), 100));
-            statement.setString(3, StringUtils.left(user.getUserGroup(), 100));
-            statement.setString(6, StringUtils.left(user.getJustification(), 500));
-            statement.setString(7, user.getStatus().toString());
-            statement.setString(8, user.getId());
-
-            // set the last accessed time accordingly
-            if (user.getLastAccessed() == null) {
-                statement.setNull(4, Types.TIMESTAMP);
-            } else {
-                statement.setTimestamp(4, new java.sql.Timestamp(user.getLastAccessed().getTime()));
-            }
-
-            // set the last verified time accordingly
-            if (user.getLastVerified() == null) {
-                statement.setNull(5, Types.TIMESTAMP);
-            } else {
-                statement.setTimestamp(5, new java.sql.Timestamp(user.getLastVerified().getTime()));
-            }
-
-            // perform the update
-            int updateCount = statement.executeUpdate();
-            if (updateCount != 1) {
-                throw new DataAccessException("Unable to update user.");
-            }
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } catch (DataAccessException dae) {
-            throw dae;
-        } finally {
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    @Override
-    public void updateGroupStatus(String group, AccountStatus status) throws DataAccessException {
-        PreparedStatement statement = null;
-        try {
-            // create a statement
-            statement = connection.prepareStatement(UPDATE_USER_GROUP_STATUS);
-            statement.setString(1, status.toString());
-            statement.setString(2, group);
-
-            // perform the update
-            statement.executeUpdate();
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } catch (DataAccessException dae) {
-            throw dae;
-        } finally {
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    @Override
-    public void updateGroupVerification(String group, Date lastVerified) throws DataAccessException {
-        PreparedStatement statement = null;
-        try {
-            // create a statement
-            statement = connection.prepareStatement(UPDATE_USER_GROUP_VERIFICATION);
-
-            // set the last verified time accordingly
-            if (lastVerified == null) {
-                statement.setNull(1, Types.TIMESTAMP);
-            } else {
-                statement.setTimestamp(1, new java.sql.Timestamp(lastVerified.getTime()));
-            }
-
-            // set the group
-            statement.setString(2, group);
-
-            // perform the update
-            statement.executeUpdate();
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } catch (DataAccessException dae) {
-            throw dae;
-        } finally {
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-    @Override
-    public void ungroup(String group) throws DataAccessException {
-        PreparedStatement statement = null;
-        try {
-            // create a statement
-            statement = connection.prepareStatement(UNGROUP_GROUP);
-            statement.setString(1, group);
-
-            // perform the update
-            statement.executeUpdate();
-        } catch (SQLException sqle) {
-            throw new DataAccessException(sqle);
-        } catch (DataAccessException dae) {
-            throw dae;
-        } finally {
-            RepositoryUtils.closeQuietly(statement);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountDisabledException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountDisabledException.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountDisabledException.java
deleted file mode 100644
index e8b3d10..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountDisabledException.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.admin.service;
-
-/**
- * Exception to indicate that the user account is disabled.
- */
-public class AccountDisabledException extends RuntimeException {
-
-    public AccountDisabledException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
-        super(message, cause, enableSuppression, writableStackTrace);
-    }
-
-    public AccountDisabledException(Throwable cause) {
-        super(cause);
-    }
-
-    public AccountDisabledException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public AccountDisabledException(String message) {
-        super(message);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountNotFoundException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountNotFoundException.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountNotFoundException.java
deleted file mode 100644
index 88287ce..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountNotFoundException.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.admin.service;
-
-/**
- * Exception to indicate that the user account is disabled.
- */
-public class AccountNotFoundException extends RuntimeException {
-
-    public AccountNotFoundException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
-        super(message, cause, enableSuppression, writableStackTrace);
-    }
-
-    public AccountNotFoundException(Throwable cause) {
-        super(cause);
-    }
-
-    public AccountNotFoundException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public AccountNotFoundException(String message) {
-        super(message);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountPendingException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountPendingException.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountPendingException.java
deleted file mode 100644
index dacc483..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountPendingException.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.admin.service;
-
-/**
- * Exception to indicate that the user has already submitting an account request
- * and that request is still pending.
- */
-public class AccountPendingException extends RuntimeException {
-
-    public AccountPendingException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
-        super(message, cause, enableSuppression, writableStackTrace);
-    }
-
-    public AccountPendingException(Throwable cause) {
-        super(cause);
-    }
-
-    public AccountPendingException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public AccountPendingException(String message) {
-        super(message);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AdministrationException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AdministrationException.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AdministrationException.java
deleted file mode 100644
index c0e8ac1..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AdministrationException.java
+++ /dev/null
@@ -1,39 +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.admin.service;
-
-/**
- *
- */
-public class AdministrationException extends RuntimeException {
-
-    public AdministrationException(Throwable cause) {
-        super(cause);
-    }
-
-    public AdministrationException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public AdministrationException(String message) {
-        super(message);
-    }
-
-    public AdministrationException() {
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AuditService.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AuditService.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AuditService.java
deleted file mode 100644
index 0843bd8..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AuditService.java
+++ /dev/null
@@ -1,76 +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.admin.service;
-
-import java.util.Collection;
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-import org.apache.nifi.action.Action;
-import org.apache.nifi.history.HistoryQuery;
-import org.apache.nifi.history.History;
-import org.apache.nifi.history.PreviousValue;
-
-/**
- * Allows NiFi actions to be audited.
- */
-public interface AuditService {
-
-    /**
-     * Adds the specified actions.
-     *
-     * @param actions
-     * @throws AdministrationException
-     */
-    void addActions(Collection<Action> actions);
-
-    /**
-     * Finds the previous values for the specified property in the specified
-     * processor. Returns null if there are none.
-     *
-     * @param processorId
-     * @return
-     */
-    Map<String, List<PreviousValue>> getPreviousValues(String processorId);
-
-    /**
-     * Get the actions within the given date range.
-     *
-     * @param actionQuery
-     * @return
-     * @throws AdministrationException
-     */
-    History getActions(HistoryQuery actionQuery);
-
-    /**
-     * Get the details for the specified action id. If the action cannot be
-     * found, null is returned.
-     *
-     * @param actionId
-     * @return
-     */
-    Action getAction(Integer actionId);
-
-    /**
-     * Purges all action's that occurred before the specified end date.
-     *
-     * @param end
-     * @param purgeAction
-     * @throws AdministrationException
-     */
-    void purgeActions(Date end, Action purgeAction);
-}


[35/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/wali/pom.xml
----------------------------------------------------------------------
diff --git a/commons/wali/pom.xml b/commons/wali/pom.xml
deleted file mode 100644
index 347c8cc..0000000
--- a/commons/wali/pom.xml
+++ /dev/null
@@ -1,41 +0,0 @@
-<?xml version="1.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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-commons-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-
-    <artifactId>wali</artifactId>
-    <version>0.0.1-SNAPSHOT</version>
-    <packaging>jar</packaging>
-    
-    <name>WALI : Write-Ahead Log Implementation</name>
-    
-    <dependencies>
-        <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>slf4j-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-utils</artifactId>
-        </dependency>
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/wali/src/main/java/org/wali/MinimalLockingWriteAheadLog.java
----------------------------------------------------------------------
diff --git a/commons/wali/src/main/java/org/wali/MinimalLockingWriteAheadLog.java b/commons/wali/src/main/java/org/wali/MinimalLockingWriteAheadLog.java
deleted file mode 100644
index 19208d3..0000000
--- a/commons/wali/src/main/java/org/wali/MinimalLockingWriteAheadLog.java
+++ /dev/null
@@ -1,1008 +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.wali;
-
-import static java.util.Objects.requireNonNull;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.channels.FileChannel;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.StandardOpenOption;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.SortedSet;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.regex.Pattern;
-
-import org.apache.nifi.stream.io.BufferedInputStream;
-import org.apache.nifi.stream.io.BufferedOutputStream;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * <p>
- * This implementation provides as little Locking as possible in order to
- * provide the highest throughput possible. However, this implementation is ONLY
- * appropriate if it can be guaranteed that only a single thread will ever issue
- * updates for a given Record at any one time.
- * </p>
- *
- * @param <T>
- */
-public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepository<T> {
-
-    private final Path basePath;
-    private final Path partialPath;
-    private final Path snapshotPath;
-
-    private final SerDe<T> serde;
-    private final SyncListener syncListener;
-    private final FileChannel lockChannel;
-    private final AtomicLong transactionIdGenerator = new AtomicLong(0L);
-
-    private final Partition<T>[] partitions;
-    private final AtomicLong partitionIndex = new AtomicLong(0L);
-    private final ConcurrentMap<Object, T> recordMap = new ConcurrentHashMap<>();
-    private final Map<Object, T> unmodifiableRecordMap = Collections.unmodifiableMap(recordMap);
-    private final Set<String> externalLocations = new CopyOnWriteArraySet<>();
-
-    private final Set<String> recoveredExternalLocations = new CopyOnWriteArraySet<>();
-
-    private final AtomicInteger numberBlackListedPartitions = new AtomicInteger(0);
-
-    private static final Logger logger = LoggerFactory.getLogger(MinimalLockingWriteAheadLog.class);
-
-    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
-    private final Lock readLock = rwLock.readLock(); // required to update a partition
-    private final Lock writeLock = rwLock.writeLock(); // required for checkpoint
-
-    private volatile boolean updated = false;
-    private volatile boolean recovered = false;
-
-    public MinimalLockingWriteAheadLog(final Path path, final int partitionCount, final SerDe<T> serde, final SyncListener syncListener) throws IOException {
-        this(new TreeSet<>(Collections.singleton(path)), partitionCount, serde, syncListener);
-    }
-
-    /**
-     *
-     * @param paths a sorted set of Paths to use for the partitions/journals and
-     * the snapshot. The snapshot will always be written to the first path
-     * specified.
-     *
-     * @param partitionCount the number of partitions/journals to use. For best
-     * performance, this should be close to the number of threads that are
-     * expected to update the repository simultaneously
-     *
-     * @param serde
-     * @param syncListener
-     * @throws IOException
-     */
-    @SuppressWarnings("unchecked")
-    public MinimalLockingWriteAheadLog(final SortedSet<Path> paths, final int partitionCount, final SerDe<T> serde, final SyncListener syncListener) throws IOException {
-        this.syncListener = syncListener;
-
-        requireNonNull(paths);
-        requireNonNull(serde);
-
-        if (paths.isEmpty()) {
-            throw new IllegalArgumentException("Paths must be non-empty");
-        }
-
-        int existingPartitions = 0;
-        for (final Path path : paths) {
-            if (!Files.exists(path)) {
-                Files.createDirectories(path);
-            }
-
-            final File file = path.toFile();
-            if (!file.isDirectory()) {
-                throw new IOException("Path given [" + path + "] is not a directory");
-            }
-            if (!file.canWrite()) {
-                throw new IOException("Path given [" + path + "] is not writable");
-            }
-            if (!file.canRead()) {
-                throw new IOException("Path given [" + path + "] is not readable");
-            }
-            if (!file.canExecute()) {
-                throw new IOException("Path given [" + path + "] is not executable");
-            }
-
-            final File[] children = file.listFiles();
-            if (children != null) {
-                for (final File child : children) {
-                    if (child.isDirectory() && child.getName().startsWith("partition-")) {
-                        existingPartitions++;
-                    }
-                }
-
-                if (existingPartitions != 0 && existingPartitions != partitionCount) {
-                    logger.warn("Constructing MinimalLockingWriteAheadLog with partitionCount={}, but the repository currently has "
-                            + "{} partitions; ignoring argument and proceeding with {} partitions",
-                            new Object[]{partitionCount, existingPartitions, existingPartitions});
-                }
-            }
-        }
-
-        this.basePath = paths.iterator().next();
-        this.partialPath = basePath.resolve("snapshot.partial");
-        this.snapshotPath = basePath.resolve("snapshot");
-        this.serde = serde;
-
-        final Path lockPath = basePath.resolve("wali.lock");
-        lockChannel = new FileOutputStream(lockPath.toFile()).getChannel();
-        lockChannel.lock();
-
-        partitions = new Partition[partitionCount];
-
-        Iterator<Path> pathIterator = paths.iterator();
-        for (int i = 0; i < partitionCount; i++) {
-            // If we're out of paths, create a new iterator to start over.
-            if (!pathIterator.hasNext()) {
-                pathIterator = paths.iterator();
-            }
-
-            final Path partitionBasePath = pathIterator.next();
-
-            partitions[i] = new Partition<>(partitionBasePath.resolve("partition-" + i), serde, i, getVersion());
-        }
-    }
-
-    @Override
-    public int update(final Collection<T> records, final boolean forceSync) throws IOException {
-        if (!recovered) {
-            throw new IllegalStateException("Cannot update repository until record recovery has been performed");
-        }
-
-        if (records.isEmpty()) {
-            return -1;
-        }
-
-        updated = true;
-        readLock.lock();
-        try {
-            while (true) {
-                final int numBlackListed = numberBlackListedPartitions.get();
-                if (numBlackListed >= partitions.length) {
-                    throw new IOException("All Partitions have been blacklisted due to failures when attempting to update. If the Write-Ahead Log is able to perform a checkpoint, this issue may resolve itself. Otherwise, manual intervention will be required.");
-                }
-
-                final long partitionIdx = partitionIndex.getAndIncrement();
-                final int resolvedIdx = (int) (partitionIdx % partitions.length);
-                final Partition<T> partition = partitions[resolvedIdx];
-                if (partition.tryClaim()) {
-                    try {
-                        final long transactionId = transactionIdGenerator.getAndIncrement();
-                        if (logger.isTraceEnabled()) {
-                            for (final T record : records) {
-                                logger.trace("Partition {} performing Transaction {}: {}", new Object[]{partition, transactionId, record});
-                            }
-                        }
-
-                        try {
-                            partition.update(records, transactionId, unmodifiableRecordMap, forceSync);
-                        } catch (final Exception e) {
-                            partition.blackList();
-                            numberBlackListedPartitions.incrementAndGet();
-                            throw e;
-                        }
-
-                        if (forceSync && syncListener != null) {
-                            syncListener.onSync(resolvedIdx);
-                        }
-                    } finally {
-                        partition.releaseClaim();
-                    }
-
-                    for (final T record : records) {
-                        final UpdateType updateType = serde.getUpdateType(record);
-                        final Object recordIdentifier = serde.getRecordIdentifier(record);
-
-                        if (updateType == UpdateType.DELETE) {
-                            recordMap.remove(recordIdentifier);
-                        } else if (updateType == UpdateType.SWAP_OUT) {
-                            final String newLocation = serde.getLocation(record);
-                            if (newLocation == null) {
-                                logger.error("Received Record (ID=" + recordIdentifier + ") with UpdateType of SWAP_OUT but no indicator of where the Record is to be Swapped Out to; these records may be lost when the repository is restored!");
-                            } else {
-                                recordMap.remove(recordIdentifier);
-                                this.externalLocations.add(newLocation);
-                            }
-                        } else if (updateType == UpdateType.SWAP_IN) {
-                            final String newLocation = serde.getLocation(record);
-                            if (newLocation == null) {
-                                logger.error("Received Record (ID=" + recordIdentifier + ") with UpdateType of SWAP_IN but no indicator of where the Record is to be Swapped In from; these records may be duplicated when the repository is restored!");
-                            } else {
-                                externalLocations.remove(newLocation);
-                            }
-                            recordMap.put(recordIdentifier, record);
-                        } else {
-                            recordMap.put(recordIdentifier, record);
-                        }
-                    }
-
-                    return resolvedIdx;
-                }
-            }
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    @Override
-    public Collection<T> recoverRecords() throws IOException {
-        if (updated) {
-            throw new IllegalStateException("Cannot recover records after updating the repository; must call recoverRecords first");
-        }
-
-        final long recoverStart = System.nanoTime();
-        writeLock.lock();
-        try {
-            Long maxTransactionId = recoverFromSnapshot(recordMap);
-            recoverFromEdits(recordMap, maxTransactionId);
-
-            for (final Partition<T> partition : partitions) {
-                final long transId = partition.getMaxRecoveredTransactionId();
-                if (maxTransactionId == null || transId > maxTransactionId) {
-                    maxTransactionId = transId;
-                }
-            }
-
-            this.transactionIdGenerator.set(maxTransactionId + 1);
-            this.externalLocations.addAll(recoveredExternalLocations);
-            logger.info("{} finished recovering records. Performing Checkpoint to ensure proper state of Partitions before updates", this);
-        } finally {
-            writeLock.unlock();
-        }
-        final long recoverNanos = System.nanoTime() - recoverStart;
-        final long recoveryMillis = TimeUnit.MILLISECONDS.convert(recoverNanos, TimeUnit.NANOSECONDS);
-        logger.info("Successfully recovered {} records in {} milliseconds", recordMap.size(), recoveryMillis);
-        checkpoint();
-
-        recovered = true;
-        return recordMap.values();
-    }
-
-    @Override
-    public Set<String> getRecoveredSwapLocations() throws IOException {
-        return recoveredExternalLocations;
-    }
-
-    private Long recoverFromSnapshot(final Map<Object, T> recordMap) throws IOException {
-        final boolean partialExists = Files.exists(partialPath);
-        final boolean snapshotExists = Files.exists(snapshotPath);
-
-        if (!partialExists && !snapshotExists) {
-            return null;
-        }
-
-        if (partialExists && snapshotExists) {
-            // both files exist -- assume we failed while checkpointing. Delete
-            // the partial file
-            Files.delete(partialPath);
-        } else if (partialExists) {
-            // partial exists but snapshot does not -- we must have completed
-            // creating the partial, deleted the snapshot
-            // but crashed before renaming the partial to the snapshot. Just
-            // rename partial to snapshot
-            Files.move(partialPath, snapshotPath);
-        }
-
-        if (Files.size(snapshotPath) == 0) {
-            logger.warn("{} Found 0-byte Snapshot file; skipping Snapshot file in recovery", this);
-            return null;
-        }
-
-        // at this point, we know the snapshotPath exists because if it didn't, then we either returned null
-        // or we renamed partialPath to snapshotPath. So just Recover from snapshotPath.
-        try (final DataInputStream dataIn = new DataInputStream(new BufferedInputStream(Files.newInputStream(snapshotPath, StandardOpenOption.READ)))) {
-            final String waliImplementationClass = dataIn.readUTF();
-            final int waliImplementationVersion = dataIn.readInt();
-
-            if (!waliImplementationClass.equals(MinimalLockingWriteAheadLog.class.getName())) {
-                throw new IOException("Write-Ahead Log located at " + snapshotPath + " was written using the " + waliImplementationClass + " class; cannot restore using " + getClass().getName());
-            }
-
-            if (waliImplementationVersion > getVersion()) {
-                throw new IOException("Write-Ahead Log located at " + snapshotPath + " was written using version " + waliImplementationVersion + " of the " + waliImplementationClass + " class; cannot restore using Version " + getVersion());
-            }
-
-            dataIn.readUTF(); // ignore serde class name for now
-            final int serdeVersion = dataIn.readInt();
-            final long maxTransactionId = dataIn.readLong();
-            final int numRecords = dataIn.readInt();
-
-            for (int i = 0; i < numRecords; i++) {
-                final T record = serde.deserializeRecord(dataIn, serdeVersion);
-                if (record == null) {
-                    throw new EOFException();
-                }
-
-                final UpdateType updateType = serde.getUpdateType(record);
-                if (updateType == UpdateType.DELETE) {
-                    logger.warn("While recovering from snapshot, found record with type 'DELETE'; this record will not be restored");
-                    continue;
-                }
-
-                logger.trace("Recovered from snapshot: {}", record);
-                recordMap.put(serde.getRecordIdentifier(record), record);
-            }
-
-            final int numSwapRecords = dataIn.readInt();
-            final Set<String> swapLocations = new HashSet<>();
-            for (int i = 0; i < numSwapRecords; i++) {
-                swapLocations.add(dataIn.readUTF());
-            }
-            this.recoveredExternalLocations.addAll(swapLocations);
-
-            logger.debug("{} restored {} Records and {} Swap Files from Snapshot, ending with Transaction ID {}", new Object[]{this, numRecords, recoveredExternalLocations.size(), maxTransactionId});
-            return maxTransactionId;
-        }
-    }
-
-    /**
-     * Recovers records from the edit logs via the Partitions. Returns a boolean
-     * if recovery of a Partition requires the Write-Ahead Log be checkpointed
-     * before modification.
-     *
-     * @param modifiableRecordMap
-     * @param maxTransactionIdRestored
-     * @return
-     * @throws IOException
-     */
-    private void recoverFromEdits(final Map<Object, T> modifiableRecordMap, final Long maxTransactionIdRestored) throws IOException {
-        final Map<Object, T> updateMap = new HashMap<>();
-        final Map<Object, T> unmodifiableRecordMap = Collections.unmodifiableMap(modifiableRecordMap);
-        final Map<Object, T> ignorableMap = new HashMap<>();
-        final Set<String> ignorableSwapLocations = new HashSet<>();
-
-        // populate a map of the next transaction id for each partition to the
-        // partition that has that next transaction id.
-        final SortedMap<Long, Partition<T>> transactionMap = new TreeMap<>();
-        for (final Partition<T> partition : partitions) {
-            Long transactionId;
-            boolean keepTransaction;
-            do {
-                transactionId = partition.getNextRecoverableTransactionId();
-
-                keepTransaction = transactionId == null || maxTransactionIdRestored == null || transactionId > maxTransactionIdRestored;
-                if (keepTransaction && transactionId != null) {
-                    // map this transaction id to its partition so that we can
-                    // start restoring transactions from this partition,
-                    // starting at 'transactionId'
-                    transactionMap.put(transactionId, partition);
-                } else if (transactionId != null) {
-                    // skip the next transaction, because our snapshot already
-                    // contained this transaction.
-                    try {
-                        partition.recoverNextTransaction(ignorableMap, updateMap, ignorableSwapLocations);
-                    } catch (final EOFException e) {
-                        logger.error("{} unexpectedly reached End of File while reading from {} for Transaction {}; assuming crash and ignoring this transaction.",
-                                new Object[]{this, partition, transactionId});
-                    }
-                }
-            } while (!keepTransaction);
-        }
-
-        while (!transactionMap.isEmpty()) {
-            final Map.Entry<Long, Partition<T>> firstEntry = transactionMap.entrySet().iterator().next();
-            final Long firstTransactionId = firstEntry.getKey();
-            final Partition<T> nextPartition = firstEntry.getValue();
-
-            try {
-                updateMap.clear();
-                final Set<Object> idsRemoved = nextPartition.recoverNextTransaction(unmodifiableRecordMap, updateMap, recoveredExternalLocations);
-                modifiableRecordMap.putAll(updateMap);
-                for (final Object id : idsRemoved) {
-                    modifiableRecordMap.remove(id);
-                }
-            } catch (final EOFException e) {
-                logger.error("{} unexpectedly reached End-of-File when reading from {} for Transaction ID {}; assuming crash and ignoring this transaction",
-                        new Object[]{this, nextPartition, firstTransactionId});
-            }
-
-            transactionMap.remove(firstTransactionId);
-
-            Long subsequentTransactionId = null;
-            try {
-                subsequentTransactionId = nextPartition.getNextRecoverableTransactionId();
-            } catch (final IOException e) {
-                logger.error("{} unexpectedly found End-of-File when reading from {} for Transaction ID {}; assuming crash and ignoring this transaction",
-                        new Object[]{this, nextPartition, firstTransactionId});
-            }
-
-            if (subsequentTransactionId != null) {
-                transactionMap.put(subsequentTransactionId, nextPartition);
-            }
-        }
-
-        for (final Partition<T> partition : partitions) {
-            partition.endRecovery();
-        }
-    }
-
-    @Override
-    public synchronized int checkpoint() throws IOException {
-        final Set<T> records;
-        final Set<String> swapLocations;
-        final long maxTransactionId;
-
-        final long startNanos = System.nanoTime();
-
-        FileOutputStream fileOut = null;
-        DataOutputStream dataOut = null;
-
-        long stopTheWorldNanos = -1L;
-        long stopTheWorldStart = -1L;
-        try {
-            writeLock.lock();
-            try {
-                stopTheWorldStart = System.nanoTime();
-                // stop the world while we make a copy of the records that must
-                // be checkpointed and rollover the partitions.
-                // We copy the records because serializing them is potentially
-                // very expensive, especially when we have hundreds
-                // of thousands or even millions of them. We don't want to
-                // prevent WALI from being used during this time.
-
-                // So the design is to copy all of the records, determine the
-                // last transaction ID that the records represent,
-                // and roll over the partitions to new write-ahead logs.
-                // Then, outside of the write lock, we will serialize the data
-                // to disk, and then remove the old Partition data.
-                records = new HashSet<>(recordMap.values());
-                maxTransactionId = transactionIdGenerator.get() - 1;
-
-                swapLocations = new HashSet<>(externalLocations);
-                for (final Partition<T> partition : partitions) {
-                    partition.rollover();
-                }
-
-                // notify global sync with the write lock held. We do this because we don't want the repository to get updated
-                // while the listener is performing its necessary tasks
-                if (syncListener != null) {
-                    syncListener.onGlobalSync();
-                }
-            } finally {
-                writeLock.unlock();
-            }
-
-            stopTheWorldNanos = System.nanoTime() - stopTheWorldStart;
-
-            // perform checkpoint, writing to .partial file
-            fileOut = new FileOutputStream(partialPath.toFile());
-            dataOut = new DataOutputStream(fileOut);
-            dataOut.writeUTF(MinimalLockingWriteAheadLog.class.getName());
-            dataOut.writeInt(getVersion());
-            dataOut.writeUTF(serde.getClass().getName());
-            dataOut.writeInt(serde.getVersion());
-            dataOut.writeLong(maxTransactionId);
-            dataOut.writeInt(records.size());
-
-            for (final T record : records) {
-                logger.trace("Checkpointing {}", record);
-                serde.serializeRecord(record, dataOut);
-            }
-
-            dataOut.writeInt(swapLocations.size());
-            for (final String swapLocation : swapLocations) {
-                dataOut.writeUTF(swapLocation);
-            }
-        } finally {
-            if (dataOut != null) {
-                try {
-                    dataOut.flush();
-                    fileOut.getFD().sync();
-                    dataOut.close();
-                } catch (final IOException e) {
-                    logger.warn("Failed to close Data Stream due to {}", e.toString(), e);
-                }
-            }
-        }
-
-        // delete the snapshot, if it exists, and rename the .partial to
-        // snapshot
-        Files.deleteIfExists(snapshotPath);
-        Files.move(partialPath, snapshotPath);
-
-        // clear all of the edit logs
-        final long partitionStart = System.nanoTime();
-        for (final Partition<T> partition : partitions) {
-            // we can call clearOld without claiming the partition because it
-            // does not change the partition's state
-            // and the only member variable it touches cannot be modified, other
-            // than when #rollover() is called.
-            // And since this method is the only one that calls #rollover() and
-            // this method is synchronized,
-            // the value of that member variable will not change. And it's
-            // volatile, so we will get the correct value.
-            partition.clearOld();
-        }
-        final long partitionEnd = System.nanoTime();
-        numberBlackListedPartitions.set(0);
-
-        final long endNanos = System.nanoTime();
-        final long millis = TimeUnit.MILLISECONDS.convert(endNanos - startNanos, TimeUnit.NANOSECONDS);
-        final long partitionMillis = TimeUnit.MILLISECONDS.convert(partitionEnd - partitionStart, TimeUnit.NANOSECONDS);
-        final long stopTheWorldMillis = TimeUnit.NANOSECONDS.toMillis(stopTheWorldNanos);
-
-        logger.info("{} checkpointed with {} Records and {} Swap Files in {} milliseconds (Stop-the-world time = {} milliseconds, Clear Edit Logs time = {} millis), max Transaction ID {}",
-                new Object[]{this, records.size(), swapLocations.size(), millis, stopTheWorldMillis, partitionMillis, maxTransactionId});
-
-        return records.size();
-    }
-
-    @Override
-    public void shutdown() throws IOException {
-        writeLock.lock();
-        try {
-            for (final Partition<T> partition : partitions) {
-                partition.close();
-            }
-        } finally {
-            writeLock.unlock();
-            lockChannel.close();
-        }
-    }
-
-    public int getVersion() {
-        return 1;
-    }
-
-    /**
-     * Represents a partition of this repository, which maps directly to a
-     * .journal file.
-     *
-     * All methods with the exceptions of {@link #claim()}, {@link #tryClaim()},
-     * and {@link #releaseClaim()} in this Partition MUST be called while
-     * holding the claim (via {@link #claim} or {@link #tryClaim()).
-     *
-     * @param <S>
-     */
-    private static class Partition<S> {
-
-        public static final String JOURNAL_EXTENSION = ".journal";
-        private static final Pattern JOURNAL_FILENAME_PATTERN = Pattern.compile("\\d+\\.journal");
-
-        private final SerDe<S> serde;
-
-        private final Path editDirectory;
-        private final int writeAheadLogVersion;
-
-        private final Lock lock = new ReentrantLock();
-        private DataOutputStream dataOut = null;
-        private FileOutputStream fileOut = null;
-        private boolean blackListed = false;
-        private boolean closed = false;
-        private DataInputStream recoveryIn;
-        private int recoveryVersion;
-        private String currentJournalFilename = "";
-
-        private static final byte TRANSACTION_CONTINUE = 1;
-        private static final byte TRANSACTION_COMMIT = 2;
-
-        private final String description;
-        private final AtomicLong maxTransactionId = new AtomicLong(-1L);
-        private final Logger logger = LoggerFactory.getLogger(MinimalLockingWriteAheadLog.class);
-
-        private final Queue<Path> recoveryFiles;
-
-        public Partition(final Path path, final SerDe<S> serde, final int partitionIndex, final int writeAheadLogVersion) throws IOException {
-            this.editDirectory = path;
-            this.serde = serde;
-
-            final File file = path.toFile();
-            if (!file.exists() && !file.mkdirs()) {
-                throw new IOException("Could not create directory " + file.getAbsolutePath());
-            }
-
-            this.recoveryFiles = new LinkedBlockingQueue<>();
-            for (final Path recoveryPath : getRecoveryPaths()) {
-                recoveryFiles.add(recoveryPath);
-            }
-
-            this.description = "Partition-" + partitionIndex;
-            this.writeAheadLogVersion = writeAheadLogVersion;
-        }
-
-        public boolean tryClaim() {
-            final boolean obtainedLock = lock.tryLock();
-            if (!obtainedLock) {
-                return false;
-            }
-
-            // Check if the partition is blacklisted. If so, unlock it and return false. Otherwise,
-            // leave it locked and return true, so that the caller will need to unlock.
-            if (blackListed) {
-                lock.unlock();
-                return false;
-            }
-
-            return true;
-        }
-
-        public void releaseClaim() {
-            lock.unlock();
-        }
-
-        public void close() {
-            final DataOutputStream out = dataOut;
-            if (out != null) {
-                try {
-                    out.close();
-                } catch (final Exception e) {
-
-                }
-            }
-
-            this.closed = true;
-            this.dataOut = null;
-        }
-
-        public void blackList() {
-            lock.lock();
-            try {
-                blackListed = true;
-            } finally {
-                lock.unlock();
-            }
-            logger.debug("Blacklisted {}", this);
-        }
-
-        /**
-         * Closes resources pointing to the current journal and begins writing
-         * to a new one
-         *
-         * @throws IOException
-         */
-        public void rollover() throws IOException {
-            lock.lock();
-            try {
-                final DataOutputStream out = dataOut;
-                if (out != null) {
-                    out.close();
-                }
-
-                final Path editPath = getNewEditPath();
-                final FileOutputStream fos = new FileOutputStream(editPath.toFile());
-                final DataOutputStream outStream = new DataOutputStream(new BufferedOutputStream(fos));
-                outStream.writeUTF(MinimalLockingWriteAheadLog.class.getName());
-                outStream.writeInt(writeAheadLogVersion);
-                outStream.writeUTF(serde.getClass().getName());
-                outStream.writeInt(serde.getVersion());
-                outStream.flush();
-                dataOut = outStream;
-                fileOut = fos;
-
-                currentJournalFilename = editPath.toFile().getName();
-
-                blackListed = false;
-            } finally {
-                lock.unlock();
-            }
-        }
-
-        private long getJournalIndex(final File file) {
-            final String filename = file.getName();
-            final int dotIndex = filename.indexOf(".");
-            final String number = filename.substring(0, dotIndex);
-            return Long.parseLong(number);
-        }
-
-        private Path getNewEditPath() {
-            final List<Path> recoveryPaths = getRecoveryPaths();
-            final long newIndex;
-            if (recoveryPaths == null || recoveryPaths.isEmpty()) {
-                newIndex = 1;
-            } else {
-                final long lastFileIndex = getJournalIndex(recoveryPaths.get(recoveryPaths.size() - 1).toFile());
-                newIndex = lastFileIndex + 1;
-            }
-
-            return editDirectory.resolve(newIndex + JOURNAL_EXTENSION);
-        }
-
-        private List<Path> getRecoveryPaths() {
-            final List<Path> paths = new ArrayList<>();
-
-            final File directory = editDirectory.toFile();
-            final File[] partitionFiles = directory.listFiles();
-            if (partitionFiles == null) {
-                return paths;
-            }
-
-            for (final File file : partitionFiles) {
-                // if file is a journal file but no data has yet been persisted, it may
-                // very well be a 0-byte file (the journal is not SYNC'ed to disk after
-                // a header is written out, so it may be lost). In this case, the journal
-                // is empty, so we can just skip it.
-                if (file.isDirectory() || file.length() == 0L) {
-                    continue;
-                }
-
-                if (!JOURNAL_FILENAME_PATTERN.matcher(file.getName()).matches()) {
-                    continue;
-                }
-
-                if (isJournalFile(file)) {
-                    paths.add(file.toPath());
-                } else {
-                    logger.warn("Found file {}, but could not access it, or it was not in the expected format; will ignore this file", file.getAbsolutePath());
-                }
-            }
-
-            // Sort journal files by the numeric portion of the filename
-            Collections.sort(paths, new Comparator<Path>() {
-                @Override
-                public int compare(final Path o1, final Path o2) {
-                    if (o1 == null && o2 == null) {
-                        return 0;
-                    }
-                    if (o1 == null) {
-                        return 1;
-                    }
-                    if (o2 == null) {
-                        return -1;
-                    }
-
-                    final long index1 = getJournalIndex(o1.toFile());
-                    final long index2 = getJournalIndex(o2.toFile());
-                    return Long.compare(index1, index2);
-                }
-            });
-
-            return paths;
-        }
-
-        void clearOld() {
-            final List<Path> oldRecoveryFiles = getRecoveryPaths();
-
-            for (final Path path : oldRecoveryFiles) {
-                final File file = path.toFile();
-                if (file.getName().equals(currentJournalFilename)) {
-                    continue;
-                }
-                if (file.exists()) {
-                    file.delete();
-                }
-            }
-        }
-
-        private boolean isJournalFile(final File file) {
-            final String expectedStartsWith = MinimalLockingWriteAheadLog.class.getName();
-            try {
-                try (final FileInputStream fis = new FileInputStream(file);
-                        final InputStream bufferedIn = new BufferedInputStream(fis);
-                        final DataInputStream in = new DataInputStream(bufferedIn)) {
-                    final String waliImplClassName = in.readUTF();
-                    if (!expectedStartsWith.equals(waliImplClassName)) {
-                        return false;
-                    }
-                }
-            } catch (final IOException e) {
-                return false;
-            }
-
-            return true;
-        }
-
-        public void update(final Collection<S> records, final long transactionId, final Map<Object, S> recordMap, final boolean forceSync) throws IOException {
-            if (this.closed) {
-                throw new IllegalStateException("Partition is closed");
-            }
-
-            final DataOutputStream out = dataOut;
-            out.writeLong(transactionId);
-
-            final int numEditsToSerialize = records.size();
-            int editsSerialized = 0;
-            for (final S record : records) {
-                final Object recordId = serde.getRecordIdentifier(record);
-                final S previousVersion = recordMap.get(recordId);
-
-                serde.serializeEdit(previousVersion, record, out);
-                if (++editsSerialized < numEditsToSerialize) {
-                    out.write(TRANSACTION_CONTINUE);
-                } else {
-                    out.write(TRANSACTION_COMMIT);
-                }
-            }
-
-            out.flush();
-
-            if (forceSync) {
-                fileOut.getFD().sync();
-            }
-        }
-
-        private DataInputStream createDataInputStream(final Path path) throws IOException {
-            return new DataInputStream(new BufferedInputStream(Files.newInputStream(path)));
-        }
-
-        private DataInputStream getRecoveryStream() throws IOException {
-            if (recoveryIn != null && hasMoreData(recoveryIn)) {
-                return recoveryIn;
-            }
-
-            while (true) {
-                final Path nextRecoveryPath = recoveryFiles.poll();
-                if (nextRecoveryPath == null) {
-                    return null;
-                }
-
-                recoveryIn = createDataInputStream(nextRecoveryPath);
-                if (hasMoreData(recoveryIn)) {
-                    final String waliImplementationClass = recoveryIn.readUTF();
-                    if (!MinimalLockingWriteAheadLog.class.getName().equals(waliImplementationClass)) {
-                        continue;
-                    }
-
-                    final long waliVersion = recoveryIn.readInt();
-                    if (waliVersion > writeAheadLogVersion) {
-                        throw new IOException("Cannot recovery from file " + nextRecoveryPath + " because it was written using WALI version " + waliVersion + ", but the version used to restore it is only " + writeAheadLogVersion);
-                    }
-
-                    @SuppressWarnings("unused")
-                    final String serdeClassName = recoveryIn.readUTF();
-                    this.recoveryVersion = recoveryIn.readInt();
-
-                    break;
-                }
-            }
-
-            return recoveryIn;
-        }
-
-        public Long getNextRecoverableTransactionId() throws IOException {
-            while (true) {
-                DataInputStream recoveryStream = getRecoveryStream();
-                if (recoveryStream == null) {
-                    return null;
-                }
-
-                final long transactionId;
-                try {
-                    transactionId = recoveryIn.readLong();
-                } catch (final EOFException e) {
-                    continue;
-                }
-
-                this.maxTransactionId.set(transactionId);
-                return transactionId;
-            }
-        }
-
-        private boolean hasMoreData(final InputStream in) throws IOException {
-            in.mark(1);
-            final int nextByte = in.read();
-            in.reset();
-            return nextByte >= 0;
-        }
-
-        public void endRecovery() throws IOException {
-            if (recoveryIn != null) {
-                recoveryIn.close();
-            }
-
-            final Path nextRecoveryPath = this.recoveryFiles.poll();
-            if (nextRecoveryPath != null) {
-                throw new IllegalStateException("Signaled to end recovery, but there are more recovery files for Partition in directory " + editDirectory);
-            }
-
-            final Path newEditPath = getNewEditPath();
-
-            final FileOutputStream fos = new FileOutputStream(newEditPath.toFile());
-            final DataOutputStream outStream = new DataOutputStream(new BufferedOutputStream(fos));
-            outStream.writeUTF(MinimalLockingWriteAheadLog.class.getName());
-            outStream.writeInt(writeAheadLogVersion);
-            outStream.writeUTF(serde.getClass().getName());
-            outStream.writeInt(serde.getVersion());
-            outStream.flush();
-            dataOut = outStream;
-            fileOut = fos;
-        }
-
-        public Set<Object> recoverNextTransaction(final Map<Object, S> currentRecordMap, final Map<Object, S> updatedRecordMap, final Set<String> swapLocations) throws IOException {
-            final Set<Object> idsRemoved = new HashSet<>();
-
-            int transactionFlag;
-            do {
-                final S record = serde.deserializeEdit(recoveryIn, currentRecordMap, recoveryVersion);
-                if (logger.isTraceEnabled()) {
-                    logger.trace("{} Recovering Transaction {}: {}", new Object[]{this, maxTransactionId.get(), record});
-                }
-
-                final Object recordId = serde.getRecordIdentifier(record);
-                final UpdateType updateType = serde.getUpdateType(record);
-                if (updateType == UpdateType.DELETE) {
-                    updatedRecordMap.remove(recordId);
-                    idsRemoved.add(recordId);
-                } else if (updateType == UpdateType.SWAP_IN) {
-                    final String location = serde.getLocation(record);
-                    if (location == null) {
-                        logger.error("Recovered SWAP_IN record from edit log, but it did not contain a Location; skipping record");
-                    } else {
-                        swapLocations.remove(location);
-                        updatedRecordMap.put(recordId, record);
-                        idsRemoved.remove(recordId);
-                    }
-                } else if (updateType == UpdateType.SWAP_OUT) {
-                    final String location = serde.getLocation(record);
-                    if (location == null) {
-                        logger.error("Recovered SWAP_OUT record from edit log, but it did not contain a Location; skipping record");
-                    } else {
-                        swapLocations.add(location);
-                        updatedRecordMap.remove(recordId);
-                        idsRemoved.add(recordId);
-                    }
-                } else {
-                    updatedRecordMap.put(recordId, record);
-                    idsRemoved.remove(recordId);
-                }
-
-                transactionFlag = recoveryIn.read();
-            } while (transactionFlag != TRANSACTION_COMMIT);
-
-            return idsRemoved;
-        }
-
-        /**
-         * Must be called after recovery has finished
-         *
-         * @return
-         */
-        public long getMaxRecoveredTransactionId() {
-            return maxTransactionId.get();
-        }
-
-        @Override
-        public String toString() {
-            return description;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/wali/src/main/java/org/wali/SerDe.java
----------------------------------------------------------------------
diff --git a/commons/wali/src/main/java/org/wali/SerDe.java b/commons/wali/src/main/java/org/wali/SerDe.java
deleted file mode 100644
index bbc7efb..0000000
--- a/commons/wali/src/main/java/org/wali/SerDe.java
+++ /dev/null
@@ -1,128 +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.wali;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.Map;
-
-/**
- * A mechanism for Serializing and De-Serializing a Record of a given Type
- *
- * @param <T> the type of record that is to be Serialized and De-Serialized by
- * this object
- */
-public interface SerDe<T> {
-
-    /**
-     * <p>
-     * Serializes an Edit Record to the log via the given
-     * {@link DataOutputStream}.
-     * </p>
-     *
-     * @param previousRecordState
-     * @param newRecordState
-     * @param out
-     * @throws IOException
-     */
-    void serializeEdit(T previousRecordState, T newRecordState, DataOutputStream out) throws IOException;
-
-    /**
-     * <p>
-     * Serializes a Record in a form suitable for a Snapshot via the given
-     * {@link DataOutputStream}.
-     * </p>
-     *
-     * @param record
-     * @param out
-     * @throws IOException
-     */
-    void serializeRecord(T record, DataOutputStream out) throws IOException;
-
-    /**
-     * <p>
-     * Reads an Edit Record from the given {@link DataInputStream} and merges
-     * that edit with the current version of the record, returning the new,
-     * merged version. If the Edit Record indicates that the entity was deleted,
-     * must return a Record with an UpdateType of {@link UpdateType#DELETE}.
-     * This method must never return <code>null</code>.
-     * </p>
-     *
-     * @param in
-     * @param currentRecordStates an unmodifiable map of Record ID's to the
-     * current state of that record
-     * @param version the version of the SerDe that was used to serialize the
-     * edit record
-     * @return
-     * @throws IOException
-     */
-    T deserializeEdit(DataInputStream in, Map<Object, T> currentRecordStates, int version) throws IOException;
-
-    /**
-     * <p>
-     * Reads a Record from the given {@link DataInputStream} and returns this
-     * record. If no data is available, returns <code>null</code>.
-     * </p>
-     *
-     * @param in
-     * @param version the version of the SerDe that was used to serialize the
-     * record
-     * @return
-     * @throws IOException
-     */
-    T deserializeRecord(DataInputStream in, int version) throws IOException;
-
-    /**
-     * Returns the unique ID for the given record
-     *
-     * @param record
-     * @return
-     */
-    Object getRecordIdentifier(T record);
-
-    /**
-     * Returns the UpdateType for the given record
-     *
-     * @param record
-     * @return
-     */
-    UpdateType getUpdateType(T record);
-
-    /**
-     * Returns the external location of the given record; this is used when a
-     * record is moved away from WALI or is being re-introduced to WALI. For
-     * example, WALI can be updated with a record of type
-     * {@link UpdateType#SWAP_OUT} that indicates a Location of
-     * file://tmp/external1 and can then be re-introduced to WALI by updating
-     * WALI with a record of type {@link UpdateType#CREATE} that indicates a
-     * Location of file://tmp/external1
-     *
-     * @param record
-     * @return
-     */
-    String getLocation(T record);
-
-    /**
-     * Returns the version that this SerDe will use when writing. This used used
-     * when serializing/deserializing the edit logs so that if the version
-     * changes, we are still able to deserialize old versions
-     *
-     * @return
-     */
-    int getVersion();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/wali/src/main/java/org/wali/SyncListener.java
----------------------------------------------------------------------
diff --git a/commons/wali/src/main/java/org/wali/SyncListener.java b/commons/wali/src/main/java/org/wali/SyncListener.java
deleted file mode 100644
index ffb11ca..0000000
--- a/commons/wali/src/main/java/org/wali/SyncListener.java
+++ /dev/null
@@ -1,62 +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.wali;
-
-/**
- * <p>
- * Provides a callback mechanism by which applicable listeners can be notified
- * when a WriteAheadRepository is synched (via the
- * {@link WriteAheadRepository#sync()} method) or one of its partitions is
- * synched via
- * {@link WriteAheadRepository#update(java.util.Collection, boolean)} with a
- * value of <code>true</code> for the second argument.
- * </p>
- *
- * <p>
- * It is not required that an implementation of {@link WriteAheadRepository}
- * support this interface. Those that do generally will require that the
- * listener be injected via the constructor.
- * </p>
- *
- * <p>
- * All implementations of this interface must be thread-safe.
- * </p>
- *
- * <p>
- * The {@link #onSync(int)} method will always be called while the associated
- * partition is locked. The {@link #onGlobalSync()} will always be called while
- * the entire repository is locked.
- * </p>
- *
- */
-public interface SyncListener {
-
-    /**
-     * This method is called whenever a specific partition is synched via the
-     * {@link WriteAheadRepository#update(java.util.Collection, boolean)} method
-     *
-     * @param partitionIndex the index of the partition that was synched
-     */
-    void onSync(int partitionIndex);
-
-    /**
-     * This method is called whenever the entire
-     * <code>WriteAheadRepository</code> is synched via the
-     * {@link WriteAheadRepository#sync()} method.
-     */
-    void onGlobalSync();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/wali/src/main/java/org/wali/UpdateType.java
----------------------------------------------------------------------
diff --git a/commons/wali/src/main/java/org/wali/UpdateType.java b/commons/wali/src/main/java/org/wali/UpdateType.java
deleted file mode 100644
index 1b039f8..0000000
--- a/commons/wali/src/main/java/org/wali/UpdateType.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.wali;
-
-/**
- * <p>
- * Enumerates the valid types of things that can cause a
- * {@link WriteAheadRepository} to update its state</p>
- */
-public enum UpdateType {
-
-    /**
-     * Used when a new Record has been created
-     */
-    CREATE,
-    /**
-     * Used when a Record has been updated in some way
-     */
-    UPDATE,
-    /**
-     * Used to indicate that a Record has been deleted and should be removed
-     * from the Repository
-     */
-    DELETE,
-    /**
-     * Used to indicate that a Record still exists but has been moved elsewhere,
-     * so that it is no longer maintained by the WALI instance
-     */
-    SWAP_OUT,
-    /**
-     * Used to indicate that a Record that was previously Swapped Out is now
-     * being Swapped In
-     */
-    SWAP_IN;
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/wali/src/main/java/org/wali/WriteAheadRepository.java
----------------------------------------------------------------------
diff --git a/commons/wali/src/main/java/org/wali/WriteAheadRepository.java b/commons/wali/src/main/java/org/wali/WriteAheadRepository.java
deleted file mode 100644
index 4567872..0000000
--- a/commons/wali/src/main/java/org/wali/WriteAheadRepository.java
+++ /dev/null
@@ -1,122 +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.wali;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Set;
-
-/**
- * <p>
- * A WriteAheadRepository is used to persist state that is otherwise kept
- * in-memory. The Repository does not provide any query capability except to
- * allow the data to be recovered upon restart of the system.
- * </p>
- *
- * <p>
- * A WriteAheadRepository operates by writing every update to an Edit Log. On
- * restart, the data can be recovered by replaying all of the updates that are
- * found in the Edit Log. This can, however, eventually result in very large
- * Edit Logs, which can both take up massive amounts of disk space and take a
- * long time to recover. In order to prevent this, the Repository provides a
- * Checkpointing capability. This allows the current in-memory state of the
- * Repository to be flushed to disk and the Edit Log to be deleted, thereby
- * compacting the amount of space required to store the Repository. After a
- * Checkpoint is performed, modifications are again written to an Edit Log. At
- * this point, when the system is to be restored, it is restored by first
- * loading the Checkpointed version of the Repository and then replaying the
- * Edit Log.
- * </p>
- *
- * <p>
- * All implementations of <code>WriteAheadRepository</code> use one or more
- * partitions to manage their Edit Logs. An implementation may require exactly
- * one partition or may allow many partitions.
- * </p>
- *
- * @param <T>
- */
-public interface WriteAheadRepository<T> {
-
-    /**
-     * <p>
-     * Updates the repository with the specified Records. The Collection must
-     * not contain multiple records with the same ID
-     * </p>
-     *
-     * @param records the records to update
-     * @param forceSync specifies whether or not the Repository forces the data
-     * to be flushed to disk. If false, the data may be stored in Operating
-     * System buffers, which improves performance but could cause loss of data
-     * if power is lost or the Operating System crashes
-     * @throws IOException
-     * @throws IllegalArgumentException if multiple records within the given
-     * Collection have the same ID, as specified by {@link Record#getId()}
-     * method
-     *
-     * @return the index of the Partition that performed the update
-     */
-    int update(Collection<T> records, boolean forceSync) throws IOException;
-
-    /**
-     * <p>
-     * Recovers all records from the persisted state. This method must be called
-     * before any updates are issued to the Repository.
-     * </p>
-     *
-     * @return
-     * @throws IOException
-     * @throws IllegalStateException if any updates have been issued against
-     * this Repository before this method is invoked
-     */
-    Collection<T> recoverRecords() throws IOException;
-
-    /**
-     * <p>
-     * Recovers all External Swap locations that were persisted. If this method
-     * is to be called, it must be called AFTER {@link #recoverRecords()} and
-     * BEFORE {@link update}.
-     * </p>
-     *
-     * @return
-     * @throws IOException
-     */
-    Set<String> getRecoveredSwapLocations() throws IOException;
-
-    /**
-     * <p>
-     * Compacts the contents of the Repository so that rather than having a
-     * Snapshot and an Edit Log indicating many Updates to the Snapshot, the
-     * Snapshot is updated to contain the current state of the Repository, and
-     * the edit log is purged.
-     * </p>
-     *
-     *
-     * @return the number of records that were written to the new snapshot
-     * @throws java.io.IOException
-     */
-    int checkpoint() throws IOException;
-
-    /**
-     * <p>
-     * Causes the repository to checkpoint and then close any open resources.
-     * </p>
-     *
-     * @throws IOException
-     */
-    void shutdown() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/wali/src/test/java/org/wali/DummyRecord.java
----------------------------------------------------------------------
diff --git a/commons/wali/src/test/java/org/wali/DummyRecord.java b/commons/wali/src/test/java/org/wali/DummyRecord.java
deleted file mode 100644
index e0f7f96..0000000
--- a/commons/wali/src/test/java/org/wali/DummyRecord.java
+++ /dev/null
@@ -1,61 +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.wali;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-public class DummyRecord {
-
-    private final String id;
-    private final Map<String, String> props;
-    private final UpdateType updateType;
-
-    public DummyRecord(final String id, final UpdateType updateType) {
-        this.id = id;
-        this.props = new HashMap<>();
-        this.updateType = updateType;
-    }
-
-    public String getId() {
-        return id;
-    }
-
-    public UpdateType getUpdateType() {
-        return updateType;
-    }
-
-    public DummyRecord setProperties(final Map<String, String> props) {
-        this.props.clear();
-        this.props.putAll(props);
-        return this;
-    }
-
-    public DummyRecord setProperty(final String name, final String value) {
-        this.props.put(name, value);
-        return this;
-    }
-
-    public Map<String, String> getProperties() {
-        return Collections.unmodifiableMap(this.props);
-    }
-
-    public String getProperty(final String name) {
-        return props.get(name);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/wali/src/test/java/org/wali/DummyRecordSerde.java
----------------------------------------------------------------------
diff --git a/commons/wali/src/test/java/org/wali/DummyRecordSerde.java b/commons/wali/src/test/java/org/wali/DummyRecordSerde.java
deleted file mode 100644
index 8cc7860..0000000
--- a/commons/wali/src/test/java/org/wali/DummyRecordSerde.java
+++ /dev/null
@@ -1,107 +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.wali;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.util.Map;
-
-public class DummyRecordSerde implements SerDe<DummyRecord> {
-
-    public static final int NUM_UPDATE_TYPES = UpdateType.values().length;
-    private int throwIOEAfterNserializeEdits = -1;
-    private int serializeEditCount = 0;
-
-    @Override
-    public void serializeEdit(final DummyRecord previousState, final DummyRecord record, final DataOutputStream out) throws IOException {
-        if (throwIOEAfterNserializeEdits >= 0 && (serializeEditCount++ >= throwIOEAfterNserializeEdits)) {
-            throw new IOException("Serialized " + (serializeEditCount - 1) + " records successfully, so now it's time to throw IOE");
-        }
-
-        out.write(record.getUpdateType().ordinal());
-        out.writeUTF(record.getId());
-
-        if (record.getUpdateType() != UpdateType.DELETE) {
-            final Map<String, String> props = record.getProperties();
-            out.writeInt(props.size());
-            for (final Map.Entry<String, String> entry : props.entrySet()) {
-                out.writeUTF(entry.getKey());
-                out.writeUTF(entry.getValue());
-            }
-        }
-    }
-
-    @Override
-    public void serializeRecord(final DummyRecord record, final DataOutputStream out) throws IOException {
-        serializeEdit(null, record, out);
-    }
-
-    @Override
-    public DummyRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
-        final int index = in.read();
-        if (index < 0) {
-            throw new EOFException();
-        }
-        if (index >= NUM_UPDATE_TYPES) {
-            throw new IOException("Corrupt stream; got UpdateType value of " + index + " but there are only " + NUM_UPDATE_TYPES + " valid values");
-        }
-        final UpdateType updateType = UpdateType.values()[index];
-        final String id = in.readUTF();
-        final DummyRecord record = new DummyRecord(id, updateType);
-
-        if (record.getUpdateType() != UpdateType.DELETE) {
-            final int numProps = in.readInt();
-            for (int i = 0; i < numProps; i++) {
-                final String key = in.readUTF();
-                final String value = in.readUTF();
-                record.setProperty(key, value);
-            }
-        }
-        return record;
-    }
-
-    @Override
-    public Object getRecordIdentifier(final DummyRecord record) {
-        return record.getId();
-    }
-
-    @Override
-    public UpdateType getUpdateType(final DummyRecord record) {
-        return record.getUpdateType();
-    }
-
-    @Override
-    public DummyRecord deserializeEdit(final DataInputStream in, final Map<Object, DummyRecord> currentVersion, final int version) throws IOException {
-        return deserializeRecord(in, version);
-    }
-
-    @Override
-    public int getVersion() {
-        return 1;
-    }
-
-    public void setThrowIOEAfterNSerializeEdits(final int n) {
-        this.throwIOEAfterNserializeEdits = n;
-    }
-
-    @Override
-    public String getLocation(final DummyRecord record) {
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/wali/src/test/java/org/wali/TestMinimalLockingWriteAheadLog.java
----------------------------------------------------------------------
diff --git a/commons/wali/src/test/java/org/wali/TestMinimalLockingWriteAheadLog.java b/commons/wali/src/test/java/org/wali/TestMinimalLockingWriteAheadLog.java
deleted file mode 100644
index 57f3495..0000000
--- a/commons/wali/src/test/java/org/wali/TestMinimalLockingWriteAheadLog.java
+++ /dev/null
@@ -1,298 +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.wali;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.FileFilter;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-public class TestMinimalLockingWriteAheadLog {
-
-    @Test
-    public void testWrite() throws IOException, InterruptedException {
-        final int numPartitions = 8;
-
-        final Path path = Paths.get("target/minimal-locking-repo");
-        deleteRecursively(path.toFile());
-        assertTrue(path.toFile().mkdirs());
-
-        final DummyRecordSerde serde = new DummyRecordSerde();
-        final WriteAheadRepository<DummyRecord> repo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null);
-        final Collection<DummyRecord> initialRecs = repo.recoverRecords();
-        assertTrue(initialRecs.isEmpty());
-
-        final List<InsertThread> threads = new ArrayList<>();
-        for (int i = 0; i < 10; i++) {
-            threads.add(new InsertThread(10000, 1000000 * i, repo));
-        }
-
-        final long start = System.nanoTime();
-        for (final InsertThread thread : threads) {
-            thread.start();
-        }
-        for (final InsertThread thread : threads) {
-            thread.join();
-        }
-        final long nanos = System.nanoTime() - start;
-        final long millis = TimeUnit.MILLISECONDS.convert(nanos, TimeUnit.NANOSECONDS);
-        System.out.println("Took " + millis + " millis to insert 1,000,000 records each in its own transaction");
-        repo.shutdown();
-
-        final WriteAheadRepository<DummyRecord> recoverRepo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null);
-        final Collection<DummyRecord> recoveredRecords = recoverRepo.recoverRecords();
-        assertFalse(recoveredRecords.isEmpty());
-        assertEquals(100000, recoveredRecords.size());
-        for (final DummyRecord record : recoveredRecords) {
-            final Map<String, String> recoveredProps = record.getProperties();
-            assertEquals(1, recoveredProps.size());
-            assertEquals("B", recoveredProps.get("A"));
-        }
-    }
-
-    @Test
-    public void testRecoverAfterIOException() throws IOException {
-        final int numPartitions = 5;
-        final Path path = Paths.get("target/minimal-locking-repo-test-recover-after-ioe");
-        deleteRecursively(path.toFile());
-        Files.createDirectories(path);
-
-        final DummyRecordSerde serde = new DummyRecordSerde();
-        final WriteAheadRepository<DummyRecord> repo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null);
-        final Collection<DummyRecord> initialRecs = repo.recoverRecords();
-        assertTrue(initialRecs.isEmpty());
-
-        serde.setThrowIOEAfterNSerializeEdits(7);   // serialize the 2 transactions, then the first edit of the third transaction; then throw IOException
-
-        final List<DummyRecord> firstTransaction = new ArrayList<>();
-        firstTransaction.add(new DummyRecord("1", UpdateType.CREATE));
-        firstTransaction.add(new DummyRecord("2", UpdateType.CREATE));
-        firstTransaction.add(new DummyRecord("3", UpdateType.CREATE));
-
-        final List<DummyRecord> secondTransaction = new ArrayList<>();
-        secondTransaction.add(new DummyRecord("1", UpdateType.UPDATE).setProperty("abc", "123"));
-        secondTransaction.add(new DummyRecord("2", UpdateType.UPDATE).setProperty("cba", "123"));
-        secondTransaction.add(new DummyRecord("3", UpdateType.UPDATE).setProperty("aaa", "123"));
-
-        final List<DummyRecord> thirdTransaction = new ArrayList<>();
-        thirdTransaction.add(new DummyRecord("1", UpdateType.DELETE));
-        thirdTransaction.add(new DummyRecord("2", UpdateType.DELETE));
-
-        repo.update(firstTransaction, true);
-        repo.update(secondTransaction, true);
-        try {
-            repo.update(thirdTransaction, true);
-            Assert.fail("Did not throw IOException on third transaction");
-        } catch (final IOException e) {
-            // expected behavior.
-        }
-
-        repo.shutdown();
-
-        serde.setThrowIOEAfterNSerializeEdits(-1);
-        final WriteAheadRepository<DummyRecord> recoverRepo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null);
-        final Collection<DummyRecord> recoveredRecords = recoverRepo.recoverRecords();
-        assertFalse(recoveredRecords.isEmpty());
-        assertEquals(3, recoveredRecords.size());
-
-        boolean record1 = false, record2 = false, record3 = false;
-        for (final DummyRecord record : recoveredRecords) {
-            switch (record.getId()) {
-                case "1":
-                    record1 = true;
-                    assertEquals("123", record.getProperty("abc"));
-                    break;
-                case "2":
-                    record2 = true;
-                    assertEquals("123", record.getProperty("cba"));
-                    break;
-                case "3":
-                    record3 = true;
-                    assertEquals("123", record.getProperty("aaa"));
-                    break;
-            }
-        }
-
-        assertTrue(record1);
-        assertTrue(record2);
-        assertTrue(record3);
-    }
-
-    @Test
-    public void testCannotModifyLogAfterAllAreBlackListed() throws IOException {
-        final int numPartitions = 5;
-        final Path path = Paths.get("target/minimal-locking-repo-test-cannot-modify-after-all-blacklisted");
-        deleteRecursively(path.toFile());
-        Files.createDirectories(path);
-
-        final DummyRecordSerde serde = new DummyRecordSerde();
-        final WriteAheadRepository<DummyRecord> repo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null);
-        final Collection<DummyRecord> initialRecs = repo.recoverRecords();
-        assertTrue(initialRecs.isEmpty());
-
-        serde.setThrowIOEAfterNSerializeEdits(3);   // serialize the first transaction, then fail on all subsequent transactions
-
-        final List<DummyRecord> firstTransaction = new ArrayList<>();
-        firstTransaction.add(new DummyRecord("1", UpdateType.CREATE));
-        firstTransaction.add(new DummyRecord("2", UpdateType.CREATE));
-        firstTransaction.add(new DummyRecord("3", UpdateType.CREATE));
-
-        final List<DummyRecord> secondTransaction = new ArrayList<>();
-        secondTransaction.add(new DummyRecord("1", UpdateType.UPDATE).setProperty("abc", "123"));
-        secondTransaction.add(new DummyRecord("2", UpdateType.UPDATE).setProperty("cba", "123"));
-        secondTransaction.add(new DummyRecord("3", UpdateType.UPDATE).setProperty("aaa", "123"));
-
-        final List<DummyRecord> thirdTransaction = new ArrayList<>();
-        thirdTransaction.add(new DummyRecord("1", UpdateType.DELETE));
-        thirdTransaction.add(new DummyRecord("2", UpdateType.DELETE));
-
-        repo.update(firstTransaction, true);
-
-        try {
-            repo.update(secondTransaction, true);
-            Assert.fail("Did not throw IOException on second transaction");
-        } catch (final IOException e) {
-            // expected behavior.
-        }
-
-        for (int i = 0; i < 4; i++) {
-            try {
-                repo.update(thirdTransaction, true);
-                Assert.fail("Did not throw IOException on third transaction");
-            } catch (final IOException e) {
-                // expected behavior.
-            }
-        }
-
-        serde.setThrowIOEAfterNSerializeEdits(-1);
-        final List<DummyRecord> fourthTransaction = new ArrayList<>();
-        fourthTransaction.add(new DummyRecord("1", UpdateType.DELETE));
-
-        try {
-            repo.update(fourthTransaction, true);
-            Assert.fail("Successfully updated repo for 4th transaction");
-        } catch (final IOException e) {
-            // expected behavior
-            assertTrue(e.getMessage().contains("All Partitions have been blacklisted"));
-        }
-
-        repo.shutdown();
-        serde.setThrowIOEAfterNSerializeEdits(-1);
-
-        final WriteAheadRepository<DummyRecord> recoverRepo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null);
-        final Collection<DummyRecord> recoveredRecords = recoverRepo.recoverRecords();
-        assertFalse(recoveredRecords.isEmpty());
-        assertEquals(3, recoveredRecords.size());
-    }
-
-    @Test
-    public void testStriping() throws IOException {
-        final int numPartitions = 6;
-        final Path path = Paths.get("target/minimal-locking-repo-striped");
-        deleteRecursively(path.toFile());
-        Files.createDirectories(path);
-
-        final SortedSet<Path> paths = new TreeSet<>();
-        paths.add(path.resolve("stripe-1"));
-        paths.add(path.resolve("stripe-2"));
-
-        final DummyRecordSerde serde = new DummyRecordSerde();
-        final WriteAheadRepository<DummyRecord> repo = new MinimalLockingWriteAheadLog<>(paths, numPartitions, serde, null);
-        final Collection<DummyRecord> initialRecs = repo.recoverRecords();
-        assertTrue(initialRecs.isEmpty());
-
-        final InsertThread inserter = new InsertThread(100000, 0, repo);
-        inserter.run();
-
-        for (final Path partitionPath : paths) {
-            final File[] files = partitionPath.toFile().listFiles(new FileFilter() {
-                @Override
-                public boolean accept(File pathname) {
-                    return pathname.getName().startsWith("partition");
-                }
-            });
-            assertEquals(3, files.length);
-
-            for (final File file : files) {
-                final File[] journalFiles = file.listFiles();
-                assertEquals(1, journalFiles.length);
-            }
-        }
-
-        repo.checkpoint();
-
-    }
-
-    private static class InsertThread extends Thread {
-
-        private final List<List<DummyRecord>> records;
-        private final WriteAheadRepository<DummyRecord> repo;
-
-        public InsertThread(final int numInsertions, final int startIndex, final WriteAheadRepository<DummyRecord> repo) {
-            records = new ArrayList<>();
-            for (int i = 0; i < numInsertions; i++) {
-                final DummyRecord record = new DummyRecord(String.valueOf(i + startIndex), UpdateType.CREATE);
-                record.setProperty("A", "B");
-                final List<DummyRecord> list = new ArrayList<>();
-                list.add(record);
-                records.add(list);
-            }
-            this.repo = repo;
-        }
-
-        @Override
-        public void run() {
-            try {
-                int counter = 0;
-                for (final List<DummyRecord> list : records) {
-                    final boolean forceSync = (++counter == records.size());
-                    repo.update(list, forceSync);
-                }
-            } catch (IOException e) {
-                Assert.fail("Failed to update: " + e.toString());
-                e.printStackTrace();
-            }
-        }
-    }
-
-    private void deleteRecursively(final File file) {
-        final File[] children = file.listFiles();
-        if (children != null) {
-            for (final File child : children) {
-                deleteRecursively(child);
-            }
-        }
-
-        file.delete();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/maven-plugins/nar-maven-plugin/pom.xml
----------------------------------------------------------------------
diff --git a/maven-plugins/nar-maven-plugin/pom.xml b/maven-plugins/nar-maven-plugin/pom.xml
new file mode 100644
index 0000000..c0236fd
--- /dev/null
+++ b/maven-plugins/nar-maven-plugin/pom.xml
@@ -0,0 +1,75 @@
+<?xml version="1.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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>maven-plugins</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+    <artifactId>nar-maven-plugin</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <packaging>maven-plugin</packaging>
+    <name>Apache NiFi NAR Plugin</name>
+    <description>Apache NiFi Nar Plugin. It is currently a part of the Apache Incubator.</description>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-plugin-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>default-descriptor</id>
+                        <goals>
+                            <goal>descriptor</goal>
+                        </goals>
+                        <phase>process-classes</phase>
+                    </execution>
+                    <execution>
+                        <id>help-descriptor</id>
+                        <goals>
+                            <goal>helpmojo</goal>
+                        </goals>
+                        <phase>process-classes</phase>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+    <dependencies>                    
+        <dependency>          
+            <groupId>org.apache.maven</groupId>
+            <artifactId>maven-plugin-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-dependency-plugin</artifactId>
+            <type>maven-plugin</type>
+        </dependency>
+        <dependency>
+            <!-- No code from maven-jar-plugin is actually used; it's included
+            just to simplify the dependencies list.                     -->
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-jar-plugin</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.maven.plugin-tools</groupId>
+            <artifactId>maven-plugin-annotations</artifactId>
+            <scope>provided</scope>
+        </dependency>
+    </dependencies>
+</project>


[18/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java
deleted file mode 100644
index dc86d24..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java
+++ /dev/null
@@ -1,171 +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.cluster.protocol.impl;
-
-import java.io.IOException;
-import java.net.Socket;
-
-import javax.net.ssl.SSLSocket;
-import javax.security.cert.X509Certificate;
-
-import org.apache.nifi.cluster.protocol.NodeProtocolSender;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller;
-import org.apache.nifi.cluster.protocol.UnknownServiceAddressException;
-import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage;
-import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage;
-import org.apache.nifi.cluster.protocol.message.ControllerStartupFailureMessage;
-import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
-import org.apache.nifi.cluster.protocol.message.NodeBulletinsMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
-import org.apache.nifi.cluster.protocol.message.ReconnectionFailureMessage;
-import org.apache.nifi.io.socket.SocketConfiguration;
-import org.apache.nifi.io.socket.SocketUtils;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-
-public class NodeProtocolSenderImpl implements NodeProtocolSender {
-    private final SocketConfiguration socketConfiguration;
-    private final ClusterServiceLocator clusterManagerProtocolServiceLocator;
-    private final ProtocolContext<ProtocolMessage> protocolContext;
-    
-    public NodeProtocolSenderImpl(final ClusterServiceLocator clusterManagerProtocolServiceLocator, 
-            final SocketConfiguration socketConfiguration, final ProtocolContext<ProtocolMessage> protocolContext) {
-        if(clusterManagerProtocolServiceLocator == null) {
-            throw new IllegalArgumentException("Protocol Service Locator may not be null.");
-        } else if(socketConfiguration == null) {
-            throw new IllegalArgumentException("Socket configuration may not be null.");
-        } else if(protocolContext == null) {
-            throw new IllegalArgumentException("Protocol Context may not be null.");
-        }
-        
-        this.clusterManagerProtocolServiceLocator = clusterManagerProtocolServiceLocator;
-        this.socketConfiguration = socketConfiguration;
-        this.protocolContext = protocolContext;
-    }
-    
-    
-    @Override
-    public ConnectionResponseMessage requestConnection(final ConnectionRequestMessage msg) throws ProtocolException, UnknownServiceAddressException {
-        Socket socket = null;
-        try {
-            socket = createSocket();
-            
-            String ncmDn = null;
-            if ( socket instanceof SSLSocket ) {
-                final SSLSocket sslSocket = (SSLSocket) socket;
-                try {
-                    final X509Certificate[] certChains = sslSocket.getSession().getPeerCertificateChain();
-                    if ( certChains != null && certChains.length > 0 ) {
-                        ncmDn = certChains[0].getSubjectDN().getName();
-                    }
-                } catch (final ProtocolException pe) {
-                    throw pe;
-                } catch (final Exception e) {
-                    throw new ProtocolException(e);
-                }
-            }
-            
-            try {
-                // marshal message to output stream
-                final ProtocolMessageMarshaller<ProtocolMessage> marshaller = protocolContext.createMarshaller();
-                marshaller.marshal(msg, socket.getOutputStream());
-            } catch(final IOException ioe) {
-                throw new ProtocolException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe);
-            } 
-            
-            final ProtocolMessage response;
-            try {
-                // unmarshall response and return
-                final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = protocolContext.createUnmarshaller();
-                response = unmarshaller.unmarshal(socket.getInputStream());
-            } catch(final IOException ioe) {
-                throw new ProtocolException("Failed unmarshalling '" + MessageType.CONNECTION_RESPONSE + "' protocol message due to: " + ioe, ioe);
-            } 
-            
-            if(MessageType.CONNECTION_RESPONSE == response.getType()) {
-                final ConnectionResponseMessage connectionResponse = (ConnectionResponseMessage) response;
-                connectionResponse.setClusterManagerDN(ncmDn);
-                return connectionResponse;
-            } else {
-                throw new ProtocolException("Expected message type '" + MessageType.CONNECTION_RESPONSE + "' but found '" + response.getType() + "'");
-            }
-        } finally {
-            SocketUtils.closeQuietly(socket);
-        }
-    }
-    
-    
-    @Override
-    public void heartbeat(final HeartbeatMessage msg) throws ProtocolException, UnknownServiceAddressException {
-        sendProtocolMessage(msg);
-    }
-
-    @Override
-    public void sendBulletins(NodeBulletinsMessage msg) throws ProtocolException, UnknownServiceAddressException {
-        sendProtocolMessage(msg);
-    }
-
-    @Override
-    public void notifyControllerStartupFailure(final ControllerStartupFailureMessage msg) throws ProtocolException, UnknownServiceAddressException {
-        sendProtocolMessage(msg);
-    }
-
-    @Override
-    public void notifyReconnectionFailure(ReconnectionFailureMessage msg) throws ProtocolException, UnknownServiceAddressException {
-        sendProtocolMessage(msg);
-    }
-    
-    private Socket createSocket() {
-        // determine the cluster manager's address
-        final DiscoverableService service = clusterManagerProtocolServiceLocator.getService(); 
-        if(service == null) {
-            throw new UnknownServiceAddressException("Cluster Manager's service is not known.  Verify a cluster manager is running.");
-        }
-        
-        try {
-            // create a socket
-            return SocketUtils.createSocket(service.getServiceAddress(), socketConfiguration); 
-        } catch(final IOException ioe) {
-            throw new ProtocolException("Failed to create socket due to: " + ioe, ioe);
-        }
-    }
-    
-    private void sendProtocolMessage(final ProtocolMessage msg) {
-        Socket socket = null;
-        try {
-            socket = createSocket();
-
-            try {
-                // marshal message to output stream
-                final ProtocolMessageMarshaller<ProtocolMessage> marshaller = protocolContext.createMarshaller();
-                marshaller.marshal(msg, socket.getOutputStream());
-            } catch(final IOException ioe) {
-                throw new ProtocolException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe);
-            }
-        } finally {
-            SocketUtils.closeQuietly(socket);
-        }
-    }
-    
-    public SocketConfiguration getSocketConfiguration() {
-        return socketConfiguration;
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java
deleted file mode 100644
index 4b359f4..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java
+++ /dev/null
@@ -1,115 +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.cluster.protocol.impl;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.nifi.cluster.protocol.NodeProtocolSender;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.ProtocolListener;
-import org.apache.nifi.cluster.protocol.UnknownServiceAddressException;
-import org.apache.nifi.cluster.protocol.message.NodeBulletinsMessage;
-import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage;
-import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage;
-import org.apache.nifi.cluster.protocol.message.ControllerStartupFailureMessage;
-import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
-import org.apache.nifi.cluster.protocol.message.ReconnectionFailureMessage;
-import org.apache.nifi.reporting.BulletinRepository;
-
-public class NodeProtocolSenderListener implements NodeProtocolSender, ProtocolListener {
-    
-    private final NodeProtocolSender sender;
-    private final ProtocolListener listener;
-    
-    public NodeProtocolSenderListener(final NodeProtocolSender sender, final ProtocolListener listener) {
-        if(sender == null) {
-            throw new IllegalArgumentException("NodeProtocolSender may not be null.");
-        } else if(listener == null) {
-            throw new IllegalArgumentException("ProtocolListener may not be null.");
-        }
-        this.sender = sender;
-        this.listener = listener;
-    }
-
-    @Override
-    public void stop() throws IOException {
-        if(!isRunning()) {
-            throw new IllegalStateException("Instance is already stopped.");
-        }
-        listener.stop();
-    }
-
-    @Override
-    public void start() throws IOException {
-        if(isRunning()) {
-            throw new IllegalStateException("Instance is already started.");
-        }
-        listener.start();
-    }
-
-    @Override
-    public boolean isRunning() {
-        return listener.isRunning();
-    }
-
-    @Override
-    public boolean removeHandler(final ProtocolHandler handler) {
-        return listener.removeHandler(handler);
-    }
-
-    @Override
-    public Collection<ProtocolHandler> getHandlers() {
-        return listener.getHandlers();
-    }
-
-    @Override
-    public void addHandler(final ProtocolHandler handler) {
-        listener.addHandler(handler);
-    }
-
-    @Override
-    public void heartbeat(final HeartbeatMessage msg) throws ProtocolException, UnknownServiceAddressException {
-        sender.heartbeat(msg);
-    }
-
-    @Override
-    public ConnectionResponseMessage requestConnection(final ConnectionRequestMessage msg) throws ProtocolException, UnknownServiceAddressException {
-        return sender.requestConnection(msg);
-    }
-    
-    @Override
-    public void notifyControllerStartupFailure(final ControllerStartupFailureMessage msg) throws ProtocolException, UnknownServiceAddressException {
-        sender.notifyControllerStartupFailure(msg);
-    }
-    
-    @Override
-    public void notifyReconnectionFailure(final ReconnectionFailureMessage msg) throws ProtocolException, UnknownServiceAddressException {
-        sender.notifyReconnectionFailure(msg);
-    }
-
-    @Override
-    public void sendBulletins(NodeBulletinsMessage msg) throws ProtocolException, UnknownServiceAddressException {
-        sender.sendBulletins(msg);
-    }
-
-    @Override
-    public void setBulletinRepository(final BulletinRepository bulletinRepository) {
-        listener.setBulletinRepository(bulletinRepository);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java
deleted file mode 100644
index ca30d9b..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java
+++ /dev/null
@@ -1,205 +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.cluster.protocol.impl;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.Socket;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.UUID;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.TimeUnit;
-
-import javax.net.ssl.SSLSocket;
-import javax.security.cert.X509Certificate;
-
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.ProtocolListener;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.events.BulletinFactory;
-import org.apache.nifi.io.socket.ServerSocketConfiguration;
-import org.apache.nifi.io.socket.SocketListener;
-import org.apache.nifi.logging.NiFiLog;
-import org.apache.nifi.reporting.Bulletin;
-import org.apache.nifi.reporting.BulletinRepository;
-import org.apache.nifi.util.StopWatch;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Implements a listener for protocol messages sent over unicast socket. 
- * 
- * @author unattributed
- */
-public class SocketProtocolListener extends SocketListener implements ProtocolListener {
-
-    private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(SocketProtocolListener.class));
-    private final ProtocolContext<ProtocolMessage> protocolContext;
-    private final Collection<ProtocolHandler> handlers = new CopyOnWriteArrayList<>();
-    private volatile BulletinRepository bulletinRepository;
-    
-    public SocketProtocolListener(
-            final int numThreads,
-            final int port,
-            final ServerSocketConfiguration configuration,
-            final ProtocolContext<ProtocolMessage> protocolContext) {
-
-        super(numThreads, port, configuration);
-        
-        if(protocolContext == null) {
-            throw new IllegalArgumentException("Protocol Context may not be null.");
-        }
-        
-        this.protocolContext = protocolContext;
-    }
-
-    @Override
-    public void setBulletinRepository(final BulletinRepository bulletinRepository) {
-        this.bulletinRepository = bulletinRepository;
-    }
-    
-    @Override
-    public void start() throws IOException {
-
-        if(super.isRunning()) {
-            throw new IllegalStateException("Instance is already started.");
-        }
-        
-        super.start();
-    }
-
-    @Override
-    public void stop() throws IOException {
-
-        if(super.isRunning() == false) {
-            throw new IOException("Instance is already stopped.");
-        }
-        
-        super.stop();
-
-    }
-
-    @Override
-    public Collection<ProtocolHandler> getHandlers() {
-        return Collections.unmodifiableCollection(handlers);
-    }
-
-    @Override
-    public void addHandler(final ProtocolHandler handler) {
-        if(handler == null) {
-            throw new NullPointerException("Protocol handler may not be null.");
-        }
-        handlers.add(handler);
-    }
-    
-    @Override
-    public boolean removeHandler(final ProtocolHandler handler) {
-        return handlers.remove(handler);
-    }
-
-    @Override
-    public void dispatchRequest(final Socket socket) {
-        byte[] receivedMessage = null;
-        String hostname = null;
-        final int maxMsgBuffer = 1024 * 1024;   // don't buffer more than 1 MB of the message
-        try {
-            final StopWatch stopWatch = new StopWatch(true);
-            hostname = socket.getInetAddress().getHostName();
-            final String requestId = UUID.randomUUID().toString();
-            logger.info("Received request {} from {}", requestId, hostname);
-            
-            String requestorDn = null;
-            if ( socket instanceof SSLSocket ) {
-                final SSLSocket sslSocket = (SSLSocket) socket;
-                try {
-                    final X509Certificate[] certChains = sslSocket.getSession().getPeerCertificateChain();
-                    if ( certChains != null && certChains.length > 0 ) {
-                        requestorDn = certChains[0].getSubjectDN().getName();
-                    }
-                } catch (final ProtocolException pe) {
-                    throw pe;
-                } catch (final Exception e) {
-                    throw new ProtocolException(e);
-                }
-            }
-            
-            // unmarshall message
-            final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = protocolContext.createUnmarshaller();
-            final InputStream inStream = socket.getInputStream();
-            final CopyingInputStream copyingInputStream = new CopyingInputStream(inStream, maxMsgBuffer); // don't copy more than 1 MB
-            logger.debug("Request {} has a message length of {}", requestId, copyingInputStream.getNumberOfBytesCopied());
-            
-            final ProtocolMessage request;
-            try {
-                request = unmarshaller.unmarshal(copyingInputStream);
-            } finally {
-                receivedMessage = copyingInputStream.getBytesRead();
-            }
-            
-            request.setRequestorDN(requestorDn);
-            
-            // dispatch message to handler
-            ProtocolHandler desiredHandler = null;
-            for (final ProtocolHandler handler : getHandlers()) {
-                if (handler.canHandle(request)) {
-                    desiredHandler = handler;
-                    break;
-                }
-            }
-
-            // if no handler found, throw exception; otherwise handle request
-            if (desiredHandler == null) {
-                throw new ProtocolException("No handler assigned to handle message type: " + request.getType());
-            } else {
-                final ProtocolMessage response = desiredHandler.handle(request);
-                if(response != null) {
-                    try {
-                        logger.debug("Sending response for request {}", requestId);
-                            
-                        // marshal message to output stream
-                        final ProtocolMessageMarshaller<ProtocolMessage> marshaller = protocolContext.createMarshaller();
-                        marshaller.marshal(response, socket.getOutputStream());
-                    } catch (final IOException ioe) {
-                        throw new ProtocolException("Failed marshalling protocol message in response to message type: " + request.getType() + " due to " + ioe, ioe);
-                    }
-                }
-            }
-            
-            stopWatch.stop();
-            logger.info("Finished processing request {} (type={}, length={} bytes) in {} millis", requestId, request.getType(), receivedMessage.length, stopWatch.getDuration(TimeUnit.MILLISECONDS));
-        } catch (final IOException e) {
-            logger.warn("Failed processing protocol message from " + hostname + " due to " + e, e);
-            
-            if ( bulletinRepository != null ) {
-                final Bulletin bulletin = BulletinFactory.createBulletin("Clustering", "WARNING", String.format("Failed to process protocol message from %s due to: %s", hostname, e.toString()));
-                bulletinRepository.addBulletin(bulletin);
-            }
-        } catch (final ProtocolException e) {
-            logger.warn("Failed processing protocol message from " + hostname + " due to " + e, e);
-            if ( bulletinRepository != null ) {
-                final Bulletin bulletin = BulletinFactory.createBulletin("Clustering", "WARNING", String.format("Failed to process protocol message from %s due to: %s", hostname, e.toString()));
-                bulletinRepository.addBulletin(bulletin);
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/JaxbProtocolContext.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/JaxbProtocolContext.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/JaxbProtocolContext.java
deleted file mode 100644
index bc68630..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/JaxbProtocolContext.java
+++ /dev/null
@@ -1,148 +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.cluster.protocol.jaxb;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
-
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller;
-
-/**
- * Implements a context for communicating internally amongst the cluster using
- * JAXB.
- * 
- * @param <T> The type of protocol message.
- *
- * @author unattributed
- */
-public class JaxbProtocolContext<T> implements ProtocolContext {
-
-    private static final int BUF_SIZE = (int) Math.pow(2, 10);  // 1k
-    
-    /*
-     * A sentinel is used to detect corrupted messages.  Relying on the integrity
-     * of the message size can cause memory issues if the value is corrupted 
-     * and equal to a number larger than the memory size.
-     */
-    private static final byte MESSAGE_PROTOCOL_START_SENTINEL = 0x5A;
-    
-    private final JAXBContext jaxbCtx;
-    
-    public JaxbProtocolContext(final JAXBContext jaxbCtx) {
-        this.jaxbCtx = jaxbCtx;
-    }
-    
-    @Override
-    public ProtocolMessageMarshaller<T> createMarshaller() {
-        return new ProtocolMessageMarshaller<T>() {
-
-            @Override
-            public void marshal(final T msg, final OutputStream os) throws IOException {
-
-                try {
-
-                    // marshal message to output stream
-                    final Marshaller marshaller = jaxbCtx.createMarshaller();
-                    final ByteArrayOutputStream msgBytes = new ByteArrayOutputStream();
-                    marshaller.marshal(msg, msgBytes);
-
-                    final DataOutputStream dos = new DataOutputStream(os);
-
-                    // write message protocol sentinel
-                    dos.write(MESSAGE_PROTOCOL_START_SENTINEL);
-                    
-                    // write message size in bytes
-                    dos.writeInt(msgBytes.size());
-
-                    // write message
-                    dos.write(msgBytes.toByteArray());
-
-                    dos.flush();
-
-                } catch (final JAXBException je) {
-                    throw new IOException("Failed marshalling protocol message due to: " + je, je);
-                }
-
-            }
-        };
-    }
-
-    @Override
-    public ProtocolMessageUnmarshaller<T> createUnmarshaller() {
-        return new ProtocolMessageUnmarshaller<T>() {
-
-            @Override
-            public T unmarshal(final InputStream is) throws IOException {
-
-                try {
-
-                    final DataInputStream dis = new DataInputStream(is);
-
-                    // check for the presence of the message protocol sentinel
-                    final byte sentinel = (byte) dis.read();
-                    if ( sentinel == -1 ) {
-                        throw new EOFException();
-                    }
-
-                    if(MESSAGE_PROTOCOL_START_SENTINEL != sentinel) {
-                        throw new IOException("Failed reading protocol message due to malformed header");
-                    }
-                    
-                    // read the message size
-                    final int msgBytesSize = dis.readInt();
-
-                    // read the message
-                    final ByteBuffer buffer = ByteBuffer.allocate(msgBytesSize);
-                    int totalBytesRead = 0;
-                    do {
-                        final int bytesToRead;
-                        if ((msgBytesSize - totalBytesRead) >= BUF_SIZE) {
-                            bytesToRead = BUF_SIZE;
-                        } else {
-                            bytesToRead = msgBytesSize - totalBytesRead;
-                        }
-                        totalBytesRead += dis.read(buffer.array(), totalBytesRead, bytesToRead);
-                    } while (totalBytesRead < msgBytesSize);
-
-                    // unmarshall message and return
-                    final Unmarshaller unmarshaller = jaxbCtx.createUnmarshaller();
-                    final byte[] msg = new byte[totalBytesRead];
-                    buffer.get(msg);
-                    return (T) unmarshaller.unmarshal(new ByteArrayInputStream(msg));
-
-                } catch (final JAXBException je) {
-                    throw new IOException("Failed unmarshalling protocol message due to: " + je, je);
-                }
-
-            }
-        };
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionRequest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionRequest.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionRequest.java
deleted file mode 100644
index d9de24f..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionRequest.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.cluster.protocol.jaxb.message;
-
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-
-/**
- * @author unattributed
- */
-public class AdaptedConnectionRequest {
-    
-    private NodeIdentifier nodeIdentifier;
-    
-    public AdaptedConnectionRequest() {}
-
-    @XmlJavaTypeAdapter(NodeIdentifierAdapter.class)
-    public NodeIdentifier getNodeIdentifier() {
-        return nodeIdentifier;
-    }
-
-    public void setNodeIdentifier(final NodeIdentifier nodeIdentifier) {
-        this.nodeIdentifier = nodeIdentifier;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java
deleted file mode 100644
index c7c783b..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java
+++ /dev/null
@@ -1,109 +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.cluster.protocol.jaxb.message;
-
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.StandardDataFlow;
-
-/**
- * @author unattributed
- */
-public class AdaptedConnectionResponse {
-    
-    private StandardDataFlow dataFlow;
-    private NodeIdentifier nodeIdentifier;
-    private boolean blockedByFirewall;
-    private boolean primary;
-    private int tryLaterSeconds;
-    private Integer managerRemoteInputPort;
-    private Boolean managerRemoteCommsSecure;
-    private String instanceId;
-    
-    public AdaptedConnectionResponse() {}
-
-    @XmlJavaTypeAdapter(DataFlowAdapter.class)
-    public StandardDataFlow getDataFlow() {
-        return dataFlow;
-    }
-
-    public void setDataFlow(StandardDataFlow dataFlow) {
-        this.dataFlow = dataFlow;
-    }
-
-    @XmlJavaTypeAdapter(NodeIdentifierAdapter.class)
-    public NodeIdentifier getNodeIdentifier() {
-        return nodeIdentifier;
-    }
-
-    public void setNodeIdentifier(NodeIdentifier nodeIdentifier) {
-        this.nodeIdentifier = nodeIdentifier;
-    }
-
-    public int getTryLaterSeconds() {
-        return tryLaterSeconds;
-    }
-
-    public void setTryLaterSeconds(int tryLaterSeconds) {
-        this.tryLaterSeconds = tryLaterSeconds;
-    }
-
-    public boolean isBlockedByFirewall() {
-        return blockedByFirewall;
-    }
-
-    public void setBlockedByFirewall(boolean blockedByFirewall) {
-        this.blockedByFirewall = blockedByFirewall;
-    }
-
-    public boolean isPrimary() {
-        return primary;
-    }
-
-    public void setPrimary(boolean primary) {
-        this.primary = primary;
-    }
-
-    public boolean shouldTryLater() {
-        return tryLaterSeconds > 0;
-    }
-    
-    public void setManagerRemoteInputPort(Integer managerRemoteInputPort) {
-        this.managerRemoteInputPort = managerRemoteInputPort;
-    }
-    
-    public Integer getManagerRemoteInputPort() {
-        return managerRemoteInputPort;
-    }
-    
-    public void setManagerRemoteCommsSecure(Boolean secure) {
-        this.managerRemoteCommsSecure = secure;
-    }
-    
-    public Boolean isManagerRemoteCommsSecure() {
-        return managerRemoteCommsSecure;
-    }
-    
-    public void setInstanceId(String instanceId) {
-        this.instanceId = instanceId;
-    }
-    
-    public String getInstanceId() {
-        return instanceId;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedCounter.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedCounter.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedCounter.java
deleted file mode 100644
index 89d903b..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedCounter.java
+++ /dev/null
@@ -1,56 +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.cluster.protocol.jaxb.message;
-
-/**
- * @author unattributed
- */
-public class AdaptedCounter {
-    
-    private String groupName;
-    
-    private String name;
-    
-    private long value;
-
-    public AdaptedCounter() {}
-
-    public String getGroupName() {
-        return groupName;
-    }
-
-    public void setGroupName(String counterGroupName) {
-        this.groupName = counterGroupName;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String counterName) {
-        this.name = counterName;
-    }
-
-    public long getValue() {
-        return value;
-    }
-
-    public void setValue(long value) {
-        this.value = value;
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedDataFlow.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedDataFlow.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedDataFlow.java
deleted file mode 100644
index bb97619..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedDataFlow.java
+++ /dev/null
@@ -1,64 +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.cluster.protocol.jaxb.message;
-
-/**
- * @author unattributed
- */
-public class AdaptedDataFlow {
-    
-    private byte[] flow;
-    private byte[] templates;
-    private byte[] snippets;
-    
-    private boolean autoStartProcessors;
-    
-    public AdaptedDataFlow() {}
-
-    public byte[] getFlow() {
-        return flow;
-    }
-
-    public void setFlow(byte[] flow) {
-        this.flow = flow;
-    }
-
-    public byte[] getTemplates() {
-        return templates;
-    }
-
-    public void setTemplates(byte[] templates) {
-        this.templates = templates;
-    }
-
-    public byte[] getSnippets() {
-        return snippets;
-    }
-
-    public void setSnippets(byte[] snippets) {
-        this.snippets = snippets;
-    }
-
-    public boolean isAutoStartProcessors() {
-        return autoStartProcessors;
-    }
-
-    public void setAutoStartProcessors(boolean runningAllProcessors) {
-        this.autoStartProcessors = runningAllProcessors;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java
deleted file mode 100644
index 5b9d9b7..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java
+++ /dev/null
@@ -1,66 +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.cluster.protocol.jaxb.message;
-
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-
-/**
- * @author unattributed
- */
-public class AdaptedHeartbeat {
-    
-    private NodeIdentifier nodeIdentifier;
-    private byte[] payload;
-    private boolean primary;
-    private boolean connected;
-    
-    public AdaptedHeartbeat() {}
-
-    @XmlJavaTypeAdapter(NodeIdentifierAdapter.class)
-    public NodeIdentifier getNodeIdentifier() {
-        return nodeIdentifier;
-    }
-
-    public void setNodeIdentifier(NodeIdentifier nodeIdentifier) {
-        this.nodeIdentifier = nodeIdentifier;
-    }
-    
-    public boolean isPrimary() {
-        return primary;
-    }
-
-    public void setPrimary(boolean primary) {
-        this.primary = primary;
-    }
-
-    public boolean isConnected() {
-        return connected;
-    }
-    
-    public void setConnected(boolean connected) {
-        this.connected = connected;
-    }
-    
-    public byte[] getPayload() {
-        return payload;
-    }
-
-    public void setPayload(byte[] payload) {
-        this.payload = payload;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeBulletins.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeBulletins.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeBulletins.java
deleted file mode 100644
index 98e2438..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeBulletins.java
+++ /dev/null
@@ -1,50 +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.cluster.protocol.jaxb.message;
-
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-
-/**
- * @author unattributed
- */
-public class AdaptedNodeBulletins {
-    
-    private NodeIdentifier nodeIdentifier;
-    
-    private byte[] payload;
-    
-    public AdaptedNodeBulletins() {}
-
-    @XmlJavaTypeAdapter(NodeIdentifierAdapter.class)
-    public NodeIdentifier getNodeIdentifier() {
-        return nodeIdentifier;
-    }
-
-    public void setNodeIdentifier(NodeIdentifier nodeIdentifier) {
-        this.nodeIdentifier = nodeIdentifier;
-    }
-    
-    public byte[] getPayload() {
-        return payload;
-    }
-
-    public void setPayload(byte[] payload) {
-        this.payload = payload;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeIdentifier.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeIdentifier.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeIdentifier.java
deleted file mode 100644
index 8134ea3..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeIdentifier.java
+++ /dev/null
@@ -1,76 +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.cluster.protocol.jaxb.message;
-
-/**
- * @author unattributed
- */
-public class AdaptedNodeIdentifier {
-    
-    private String id;
-    
-    private String apiAddress;
-
-    private int apiPort;    
-
-    private String socketAddress;
-    
-    private int socketPort;
-    
-    public AdaptedNodeIdentifier() {}
-
-    public String getApiAddress() {
-        return apiAddress;
-    }
-
-    public void setApiAddress(String apiAddress) {
-        this.apiAddress = apiAddress;
-    }
-
-    public int getApiPort() {
-        return apiPort;
-    }
-
-    public void setApiPort(int apiPort) {
-        this.apiPort = apiPort;
-    }
-
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    public String getSocketAddress() {
-        return socketAddress;
-    }
-
-    public void setSocketAddress(String socketAddress) {
-        this.socketAddress = socketAddress;
-    }
-
-    public int getSocketPort() {
-        return socketPort;
-    }
-
-    public void setSocketPort(int socketPort) {
-        this.socketPort = socketPort;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionRequestAdapter.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionRequestAdapter.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionRequestAdapter.java
deleted file mode 100644
index 1f91cf1..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionRequestAdapter.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.cluster.protocol.jaxb.message;
-
-import javax.xml.bind.annotation.adapters.XmlAdapter;
-import org.apache.nifi.cluster.protocol.ConnectionRequest;
-
-/**
- * @author unattributed
- */
-public class ConnectionRequestAdapter extends XmlAdapter<AdaptedConnectionRequest, ConnectionRequest> {
-
-    @Override
-    public AdaptedConnectionRequest marshal(final ConnectionRequest cr) {
-        final AdaptedConnectionRequest aCr = new AdaptedConnectionRequest();
-        if(cr != null) {
-            aCr.setNodeIdentifier(cr.getProposedNodeIdentifier());
-        }
-        return aCr;
-    }
-
-    @Override
-    public ConnectionRequest unmarshal(final AdaptedConnectionRequest aCr) {
-        return new ConnectionRequest(aCr.getNodeIdentifier());
-    }
- 
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java
deleted file mode 100644
index 143bab0..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java
+++ /dev/null
@@ -1,55 +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.cluster.protocol.jaxb.message;
-
-import javax.xml.bind.annotation.adapters.XmlAdapter;
-import org.apache.nifi.cluster.protocol.ConnectionResponse;
-
-/**
- * @author unattributed
- */
-public class ConnectionResponseAdapter extends XmlAdapter<AdaptedConnectionResponse, ConnectionResponse> {
-
-    @Override
-    public AdaptedConnectionResponse marshal(final ConnectionResponse cr) {
-        final AdaptedConnectionResponse aCr = new AdaptedConnectionResponse();
-        if(cr != null) {
-            aCr.setDataFlow(cr.getDataFlow());
-            aCr.setNodeIdentifier(cr.getNodeIdentifier());
-            aCr.setTryLaterSeconds(cr.getTryLaterSeconds());
-            aCr.setBlockedByFirewall(cr.isBlockedByFirewall());
-            aCr.setPrimary(cr.isPrimary());
-            aCr.setManagerRemoteInputPort(cr.getManagerRemoteInputPort());
-            aCr.setManagerRemoteCommsSecure(cr.isManagerRemoteCommsSecure());
-            aCr.setInstanceId(cr.getInstanceId());
-        }
-        return aCr;
-    }
-
-    @Override
-    public ConnectionResponse unmarshal(final AdaptedConnectionResponse aCr) {
-        if(aCr.shouldTryLater()) {
-            return new ConnectionResponse(aCr.getTryLaterSeconds());
-        } else if(aCr.isBlockedByFirewall()) {
-            return ConnectionResponse.createBlockedByFirewallResponse();
-        } else {
-            return new ConnectionResponse(aCr.getNodeIdentifier(), aCr.getDataFlow(), aCr.isPrimary(), 
-                aCr.getManagerRemoteInputPort(), aCr.isManagerRemoteCommsSecure(), aCr.getInstanceId());
-        }
-    }
- 
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/DataFlowAdapter.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/DataFlowAdapter.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/DataFlowAdapter.java
deleted file mode 100644
index 8d9467f..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/DataFlowAdapter.java
+++ /dev/null
@@ -1,50 +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.cluster.protocol.jaxb.message;
-
-import javax.xml.bind.annotation.adapters.XmlAdapter;
-
-import org.apache.nifi.cluster.protocol.StandardDataFlow;
-
-/**
- * @author unattributed
- */
-public class DataFlowAdapter extends XmlAdapter<AdaptedDataFlow, StandardDataFlow> {
-
-    @Override
-    public AdaptedDataFlow marshal(final StandardDataFlow df) {
-        
-        final AdaptedDataFlow aDf = new AdaptedDataFlow();
-        
-        if(df != null) {
-            aDf.setFlow(df.getFlow());
-            aDf.setTemplates(df.getTemplates());
-            aDf.setSnippets(df.getSnippets());
-            aDf.setAutoStartProcessors(df.isAutoStartProcessors());
-        }
-        
-        return aDf;
-    }
-
-    @Override
-    public StandardDataFlow unmarshal(final AdaptedDataFlow aDf) {
-        final StandardDataFlow dataFlow = new StandardDataFlow(aDf.getFlow(), aDf.getTemplates(), aDf.getSnippets());
-        dataFlow.setAutoStartProcessors(aDf.isAutoStartProcessors());
-        return dataFlow;
-    }
- 
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java
deleted file mode 100644
index 0e073b6..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java
+++ /dev/null
@@ -1,54 +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.cluster.protocol.jaxb.message;
-
-import javax.xml.bind.annotation.adapters.XmlAdapter;
-import org.apache.nifi.cluster.protocol.Heartbeat;
-
-/**
- * @author unattributed
- */
-public class HeartbeatAdapter extends XmlAdapter<AdaptedHeartbeat, Heartbeat> {
-
-    @Override
-    public AdaptedHeartbeat marshal(final Heartbeat hb) {
-        
-        final AdaptedHeartbeat aHb = new AdaptedHeartbeat();
-        
-        if(hb != null) {
-            // set node identifier
-            aHb.setNodeIdentifier(hb.getNodeIdentifier());
-
-            // set payload
-            aHb.setPayload(hb.getPayload());
-            
-            // set leader flag
-            aHb.setPrimary(hb.isPrimary());
-            
-            // set connected flag
-            aHb.setConnected(hb.isConnected());
-        }
-        
-        return aHb;
-    }
-
-    @Override
-    public Heartbeat unmarshal(final AdaptedHeartbeat aHb) {
-        return new Heartbeat(aHb.getNodeIdentifier(), aHb.isPrimary(), aHb.isConnected(), aHb.getPayload());
-    }
- 
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java
deleted file mode 100644
index c3a57f5..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.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.cluster.protocol.jaxb.message;
-
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-
-/**
- * @author unattributed
- */
-public final class JaxbProtocolUtils {
-    
-    public static final String JAXB_CONTEXT_PATH = ObjectFactory.class.getPackage().getName();
-
-    public static final JAXBContext JAXB_CONTEXT = initializeJaxbContext();
-    
-    /**
-     * Load the JAXBContext version.
-     */
-    private static JAXBContext initializeJaxbContext() {
-        try {
-            return JAXBContext.newInstance(JAXB_CONTEXT_PATH);
-        } catch (JAXBException e) {
-            throw new RuntimeException("Unable to create JAXBContext.", e);
-        }
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeBulletinsAdapter.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeBulletinsAdapter.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeBulletinsAdapter.java
deleted file mode 100644
index 1ae41f7..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeBulletinsAdapter.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.cluster.protocol.jaxb.message;
-
-import javax.xml.bind.annotation.adapters.XmlAdapter;
-import org.apache.nifi.cluster.protocol.NodeBulletins;
-
-/**
- * @author unattributed
- */
-public class NodeBulletinsAdapter extends XmlAdapter<AdaptedNodeBulletins, NodeBulletins> {
-
-    @Override
-    public AdaptedNodeBulletins marshal(final NodeBulletins hb) {
-        
-        final AdaptedNodeBulletins adaptedBulletins = new AdaptedNodeBulletins();
-        
-        if(hb != null) {
-            // set node identifier
-            adaptedBulletins.setNodeIdentifier(hb.getNodeIdentifier());
-
-            // set payload
-            adaptedBulletins.setPayload(hb.getPayload());
-        }
-        
-        return adaptedBulletins;
-    }
-
-    @Override
-    public NodeBulletins unmarshal(final AdaptedNodeBulletins adaptedBulletins) {
-        return new NodeBulletins(adaptedBulletins.getNodeIdentifier(), adaptedBulletins.getPayload());
-    }
- 
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeIdentifierAdapter.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeIdentifierAdapter.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeIdentifierAdapter.java
deleted file mode 100644
index fe2d8a4..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeIdentifierAdapter.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.cluster.protocol.jaxb.message;
-
-import javax.xml.bind.annotation.adapters.XmlAdapter;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-
-/**
- * @author unattributed
- */
-public class NodeIdentifierAdapter extends XmlAdapter<AdaptedNodeIdentifier, NodeIdentifier> {
-
-    @Override
-    public AdaptedNodeIdentifier marshal(final NodeIdentifier ni) {
-        if(ni == null) {
-            return null;
-        } else {
-            final AdaptedNodeIdentifier aNi = new AdaptedNodeIdentifier();
-            aNi.setId(ni.getId());
-            aNi.setApiAddress(ni.getApiAddress());
-            aNi.setApiPort(ni.getApiPort());
-            aNi.setSocketAddress(ni.getSocketAddress());
-            aNi.setSocketPort(ni.getSocketPort());
-            return aNi;
-        }
-    }
-
-    @Override
-    public NodeIdentifier unmarshal(final AdaptedNodeIdentifier aNi) {
-        if(aNi == null) {
-            return null;
-        } else {
-            return new NodeIdentifier(aNi.getId(), aNi.getApiAddress(), aNi.getApiPort(), aNi.getSocketAddress(), aNi.getSocketPort());
-        }
-    }
- 
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java
deleted file mode 100644
index 1613536..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java
+++ /dev/null
@@ -1,104 +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.cluster.protocol.jaxb.message;
-
-import javax.xml.bind.annotation.XmlRegistry;
-
-import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage;
-import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage;
-import org.apache.nifi.cluster.protocol.message.ControllerStartupFailureMessage;
-import org.apache.nifi.cluster.protocol.message.DisconnectMessage;
-import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
-import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
-import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
-import org.apache.nifi.cluster.protocol.message.MulticastProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.NodeBulletinsMessage;
-import org.apache.nifi.cluster.protocol.message.PingMessage;
-import org.apache.nifi.cluster.protocol.message.PrimaryRoleAssignmentMessage;
-import org.apache.nifi.cluster.protocol.message.ReconnectionFailureMessage;
-import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
-import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage;
-import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage;
-
-/**
- * @author unattributed
- */
-@XmlRegistry
-public class ObjectFactory {
-    
-    public ObjectFactory() {}
-    
-    public ReconnectionRequestMessage createReconnectionRequestMessage() {
-        return new ReconnectionRequestMessage();
-    }
-    
-    public ReconnectionFailureMessage createReconnectionFailureMessage() {
-        return new ReconnectionFailureMessage();
-    }
-    
-    public ReconnectionResponseMessage createReconnectionResponseMessage() {
-        return new ReconnectionResponseMessage();
-    }
-    
-    public DisconnectMessage createDisconnectionMessage() {
-        return new DisconnectMessage();
-    }
-    
-    public ConnectionRequestMessage createConnectionRequestMessage() {
-        return new ConnectionRequestMessage();
-    }
-    
-    public ConnectionResponseMessage createConnectionResponseMessage() {
-        return new ConnectionResponseMessage();
-    }
-    
-    public ServiceBroadcastMessage createServiceBroadcastMessage() {
-        return new ServiceBroadcastMessage();
-    }
-    
-    public HeartbeatMessage createHeartbeatMessage() {
-        return new HeartbeatMessage();
-    }
-    
-    public FlowRequestMessage createFlowRequestMessage() {
-        return new FlowRequestMessage();
-    }
-    
-    public FlowResponseMessage createFlowResponseMessage() {
-        return new FlowResponseMessage();
-    }
-    
-    public PingMessage createPingMessage() {
-        return new PingMessage();
-    }
-    
-    public MulticastProtocolMessage createMulticastProtocolMessage() {
-        return new MulticastProtocolMessage();
-    }
-    
-    public ControllerStartupFailureMessage createControllerStartupFailureMessage() {
-        return new ControllerStartupFailureMessage();
-    }
-    
-    public PrimaryRoleAssignmentMessage createPrimaryRoleAssignmentMessage() {
-        return new PrimaryRoleAssignmentMessage();
-    }
-    
-    public NodeBulletinsMessage createBulletinsMessage() {
-        return new NodeBulletinsMessage();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionRequestMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionRequestMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionRequestMessage.java
deleted file mode 100644
index 344de4e..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionRequestMessage.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.cluster.protocol.message;
-
-import javax.xml.bind.annotation.XmlRootElement;
-
-import org.apache.nifi.cluster.protocol.ConnectionRequest;
-
-/**
- * @author unattributed
- */
-@XmlRootElement(name = "connectionRequestMessage")
-public class ConnectionRequestMessage extends ProtocolMessage {
-    
-    private ConnectionRequest connectionRequest;
-    
-    public ConnectionRequestMessage() {}
-    
-    public ConnectionRequest getConnectionRequest() {
-        return connectionRequest;
-    }
-
-    public void setConnectionRequest(ConnectionRequest connectionRequest) {
-        this.connectionRequest = connectionRequest;
-    }
-    
-    @Override
-    public MessageType getType() {
-        return MessageType.CONNECTION_REQUEST;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionResponseMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionResponseMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionResponseMessage.java
deleted file mode 100644
index a262d7a..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionResponseMessage.java
+++ /dev/null
@@ -1,66 +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.cluster.protocol.message;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.cluster.protocol.ConnectionResponse;
-
-/**
- * @author unattributed
- */
-@XmlRootElement(name = "connectionResponseMessage")
-public class ConnectionResponseMessage extends ProtocolMessage {
-    
-    private ConnectionResponse connectionResponse;
-    private String clusterManagerDN;
-    
-    public ConnectionResponseMessage() {}
-
-    public ConnectionResponse getConnectionResponse() {
-        return connectionResponse;
-    }
-
-    public void setConnectionResponse(final ConnectionResponse connectionResponse) {
-        this.connectionResponse = connectionResponse;
-        
-        if ( clusterManagerDN != null ) {
-            this.connectionResponse.setClusterManagerDN(clusterManagerDN);
-        }
-    }
-    
-    public void setClusterManagerDN(final String dn) {
-        if ( connectionResponse != null ) {
-            connectionResponse.setClusterManagerDN(dn);
-        }
-        this.clusterManagerDN = dn;
-    }
-    
-    /**
-     * Returns the DN of the NCM, if it is available or <code>null</code> otherwise.
-     * 
-     * @return
-     */
-    public String getClusterManagerDN() {
-        return clusterManagerDN;
-    }
-
-    @Override
-    public MessageType getType() {
-        return MessageType.CONNECTION_RESPONSE;
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ControllerStartupFailureMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ControllerStartupFailureMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ControllerStartupFailureMessage.java
deleted file mode 100644
index ebc1cae..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ControllerStartupFailureMessage.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.cluster.protocol.message;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter;
-
-/**
- * @author unattributed
- */
-@XmlRootElement(name = "controllerStartupFailureMessage")
-public class ControllerStartupFailureMessage extends ExceptionMessage {
-
-    private NodeIdentifier nodeId;
-    
-    public ControllerStartupFailureMessage() {}
-    
-    @Override
-    public MessageType getType() {
-        return MessageType.CONTROLLER_STARTUP_FAILURE;
-    }
-
-    @XmlJavaTypeAdapter(NodeIdentifierAdapter.class)
-    public NodeIdentifier getNodeId() {
-        return nodeId;
-    }
-
-    public void setNodeId(NodeIdentifier nodeId) {
-        this.nodeId = nodeId;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/DisconnectMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/DisconnectMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/DisconnectMessage.java
deleted file mode 100644
index 8aa7a40..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/DisconnectMessage.java
+++ /dev/null
@@ -1,55 +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.cluster.protocol.message;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter;
-
-/**
- * @author unattributed
- */
-@XmlRootElement(name = "disconnectionMessage")
-public class DisconnectMessage extends ProtocolMessage {
-    
-    private NodeIdentifier nodeId;
-    private String explanation;
- 
-    @XmlJavaTypeAdapter(NodeIdentifierAdapter.class)
-    public NodeIdentifier getNodeId() {
-        return nodeId;
-    }
-
-    public void setNodeId(NodeIdentifier nodeId) {
-        this.nodeId = nodeId;
-    }
-
-    public String getExplanation() {
-        return explanation;
-    }
-
-    public void setExplanation(String explanation) {
-        this.explanation = explanation;
-    }
-    
-    @Override
-    public MessageType getType() {
-        return MessageType.DISCONNECTION_REQUEST;
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ExceptionMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ExceptionMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ExceptionMessage.java
deleted file mode 100644
index 99a6dee..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ExceptionMessage.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.cluster.protocol.message;
-
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- * @author unattributed
- */
-@XmlRootElement(name = "exceptionMessage")
-public class ExceptionMessage extends ProtocolMessage {
-    
-    private String exceptionMessage;
-
-    public ExceptionMessage() {}
-
-    public String getExceptionMessage() {
-        return exceptionMessage;
-    }
-
-    public void setExceptionMessage(String exceptionMessage) {
-        this.exceptionMessage = exceptionMessage;
-    }
-    
-    @Override
-    public MessageType getType() {
-        return MessageType.EXCEPTION;
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowRequestMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowRequestMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowRequestMessage.java
deleted file mode 100644
index 4a10538..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowRequestMessage.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.cluster.protocol.message;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter;
-
-/**
- * @author unattributed
- */
-@XmlRootElement(name = "flowRequestMessage")
-public class FlowRequestMessage extends ProtocolMessage {
-    
-    private NodeIdentifier nodeId;
-
-    @XmlJavaTypeAdapter(NodeIdentifierAdapter.class)
-    public NodeIdentifier getNodeId() {
-        return nodeId;
-    }
-
-    public void setNodeId(NodeIdentifier nodeId) {
-        this.nodeId = nodeId;
-    }
-    
-    @Override
-    public MessageType getType() {
-        return MessageType.FLOW_REQUEST;
-    }
-    
-}


[47/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
deleted file mode 100644
index 420a8e2..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
+++ /dev/null
@@ -1,1186 +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.attribute.expression.language;
-
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.*;
-
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionLexer;
-import org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.cast.BooleanCastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.cast.DateCastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.cast.NumberCastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.cast.StringCastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.AndEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.AppendEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.AttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ContainsEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.DateToNumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.DivideEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.EndsWithEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.EqualsEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.EqualsIgnoreCaseEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.FindEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.FormatEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.GreaterThanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.GreaterThanOrEqualEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.HostnameEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.IPEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.IndexOfEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.IsEmptyEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.IsNullEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.LastIndexOfEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.LengthEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.LessThanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.LessThanOrEqualEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.MatchesEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.MinusEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ModEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.MultiplyEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.NotEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.NotNullEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.NowEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.NumberToDateEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.OneUpSequenceEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.OrEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.PlusEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.PrependEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceAllEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceEmptyEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceNullEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.StartsWithEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.StringToDateEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringAfterEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringAfterLastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringBeforeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringBeforeLastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ToLowerEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ToNumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ToRadixEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ToStringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ToUpperEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.TrimEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.UrlDecodeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.UrlEncodeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.UuidEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.literals.BooleanLiteralEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.literals.NumberLiteralEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.reduce.CountEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.reduce.JoinEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.reduce.ReduceEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.AllAttributesEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.AnyAttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.DelineatedAttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiAttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiMatchAttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiNamedAttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
-import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-import org.apache.nifi.expression.AttributeValueDecorator;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.exception.ProcessException;
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CharStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.tree.Tree;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.MappingEvaluator;
-
-/**
- * Class used for creating and evaluating NiFi Expression Language. Once a Query
- * has been created, it may be evaluated using the evaluate methods exactly
- * once.
- */
-public class Query {
-
-    private final String query;
-    private final Tree tree;
-    private final Evaluator<?> evaluator;
-    private final AtomicBoolean evaluated = new AtomicBoolean(false);
-
-    private Query(final String query, final Tree tree, final Evaluator<?> evaluator) {
-        this.query = query;
-        this.tree = tree;
-        this.evaluator = evaluator;
-    }
-
-    public static boolean isValidExpression(final String value) {
-        try {
-            validateExpression(value, false);
-            return true;
-        } catch (final ProcessException e) {
-            return false;
-        }
-    }
-
-    public static ResultType getResultType(final String value) throws AttributeExpressionLanguageParsingException {
-        return Query.compile(value).getResultType();
-    }
-
-    public static List<ResultType> extractResultTypes(final String value) throws AttributeExpressionLanguageParsingException {
-        final List<ResultType> types = new ArrayList<>();
-
-        for (final Range range : extractExpressionRanges(value)) {
-            final String text = value.substring(range.getStart(), range.getEnd() + 1);
-            types.add(getResultType(text));
-        }
-
-        return types;
-    }
-
-    public static List<String> extractExpressions(final String value) throws AttributeExpressionLanguageParsingException {
-        final List<String> expressions = new ArrayList<>();
-
-        for (final Range range : extractExpressionRanges(value)) {
-            expressions.add(value.substring(range.getStart(), range.getEnd() + 1));
-        }
-
-        return expressions;
-    }
-
-    public static List<Range> extractExpressionRanges(final String value) throws AttributeExpressionLanguageParsingException {
-        final List<Range> ranges = new ArrayList<>();
-        char lastChar = 0;
-        int embeddedCount = 0;
-        int expressionStart = -1;
-        boolean oddDollarCount = false;
-        int backslashCount = 0;
-
-        charLoop:
-        for (int i = 0; i < value.length(); i++) {
-            final char c = value.charAt(i);
-
-            if (expressionStart > -1 && (c == '\'' || c == '"') && (lastChar != '\\' || backslashCount % 2 == 0)) {
-                final int endQuoteIndex = findEndQuoteChar(value, i);
-                if (endQuoteIndex < 0) {
-                    break charLoop;
-                }
-
-                i = endQuoteIndex;
-                continue;
-            }
-
-            if (c == '{') {
-                if (oddDollarCount && lastChar == '$') {
-                    if (embeddedCount == 0) {
-                        expressionStart = i - 1;
-                    }
-                }
-
-                embeddedCount++;
-            } else if (c == '}') {
-                if (embeddedCount <= 0) {
-                    continue;
-                }
-
-                if (--embeddedCount == 0) {
-                    if (expressionStart > -1) {
-                        // ended expression. Add a new range.
-                        final Range range = new Range(expressionStart, i);
-                        ranges.add(range);
-                    }
-
-                    expressionStart = -1;
-                }
-            } else if (c == '$') {
-                oddDollarCount = !oddDollarCount;
-            } else if (c == '\\') {
-                backslashCount++;
-            } else {
-                oddDollarCount = false;
-            }
-
-            lastChar = c;
-        }
-
-        return ranges;
-    }
-
-    /**
-     *
-     *
-     * @param value
-     * @param allowSurroundingCharacters
-     * @throws AttributeExpressionLanguageParsingException
-     */
-    public static void validateExpression(final String value, final boolean allowSurroundingCharacters) throws AttributeExpressionLanguageParsingException {
-        if (!allowSurroundingCharacters) {
-            final List<Range> ranges = extractExpressionRanges(value);
-            if (ranges.size() > 1) {
-                throw new AttributeExpressionLanguageParsingException("Found multiple Expressions but expected only 1");
-            }
-
-            if (ranges.isEmpty()) {
-                throw new AttributeExpressionLanguageParsingException("No Expressions found");
-            }
-
-            final Range range = ranges.get(0);
-            final String expression = value.substring(range.getStart(), range.getEnd() + 1);
-            Query.compile(expression);
-
-            if (range.getStart() > 0 || range.getEnd() < value.length() - 1) {
-                throw new AttributeExpressionLanguageParsingException("Found characters outside of Expression");
-            }
-        } else {
-            for (final Range range : extractExpressionRanges(value)) {
-                final String expression = value.substring(range.getStart(), range.getEnd() + 1);
-                Query.compile(expression);
-            }
-        }
-    }
-
-    static int findEndQuoteChar(final String value, final int quoteStart) {
-        final char quoteChar = value.charAt(quoteStart);
-
-        int backslashCount = 0;
-        char lastChar = 0;
-        for (int i = quoteStart + 1; i < value.length(); i++) {
-            final char c = value.charAt(i);
-
-            if (c == '\\') {
-                backslashCount++;
-            } else if (c == quoteChar && ((backslashCount % 2 == 0) || lastChar != '\\')) {
-                return i;
-            }
-
-            lastChar = c;
-        }
-
-        return -1;
-    }
-
-    static String evaluateExpression(final Tree tree, final String queryText, final Map<String, String> expressionMap, final AttributeValueDecorator decorator) throws ProcessException {
-        final Object evaluated = Query.fromTree(tree, queryText).evaluate(expressionMap).getValue();
-        if (evaluated == null) {
-            return null;
-        }
-
-        final String value = evaluated.toString();
-        final String escaped = value.replace("$$", "$");
-        return (decorator == null) ? escaped : decorator.decorate(escaped);
-    }
-
-    static String evaluateExpressions(final String rawValue, Map<String, String> expressionMap) throws ProcessException {
-        return evaluateExpressions(rawValue, expressionMap, null);
-    }
-
-    static String evaluateExpressions(final String rawValue) throws ProcessException {
-        return evaluateExpressions(rawValue, createExpressionMap(null), null);
-    }
-
-    static String evaluateExpressions(final String rawValue, final FlowFile flowFile) throws ProcessException {
-        return evaluateExpressions(rawValue, createExpressionMap(flowFile), null);
-    }
-
-    static String evaluateExpressions(final String rawValue, Map<String, String> expressionMap, final AttributeValueDecorator decorator) throws ProcessException {
-        return Query.prepare(rawValue).evaluateExpressions(expressionMap, decorator);
-    }
-
-    public static String evaluateExpressions(final String rawValue, final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException {
-        if (rawValue == null) {
-            return null;
-        }
-
-        final Map<String, String> expressionMap = createExpressionMap(flowFile);
-        return evaluateExpressions(rawValue, expressionMap, decorator);
-    }
-
-    private static Evaluator<?> getRootSubjectEvaluator(final Evaluator<?> evaluator) {
-        if (evaluator == null) {
-            return null;
-        }
-
-        final Evaluator<?> subject = evaluator.getSubjectEvaluator();
-        if (subject == null) {
-            return evaluator;
-        }
-
-        return getRootSubjectEvaluator(subject);
-    }
-
-    /**
-     * Un-escapes ${...} patterns that were escaped
-     *
-     * @param value
-     * @return
-     */
-    public static String unescape(final String value) {
-        return value.replaceAll("\\$\\$(?=\\$*\\{.*?\\})", "\\$");
-    }
-
-    static Map<String, String> createExpressionMap(final FlowFile flowFile) {
-        final Map<String, String> attributeMap = flowFile == null ? new HashMap<String, String>() : flowFile.getAttributes();
-        final Map<String, String> envMap = System.getenv();
-        final Map<?, ?> sysProps = System.getProperties();
-
-        final Map<String, String> flowFileProps = new HashMap<>();
-        if (flowFile != null) {
-            flowFileProps.put("flowFileId", String.valueOf(flowFile.getId()));
-            flowFileProps.put("fileSize", String.valueOf(flowFile.getSize()));
-            flowFileProps.put("entryDate", String.valueOf(flowFile.getEntryDate()));
-            flowFileProps.put("lineageStartDate", String.valueOf(flowFile.getLineageStartDate()));
-        }
-
-        return wrap(attributeMap, flowFileProps, envMap, sysProps);
-    }
-
-    private static Map<String, String> wrap(final Map<String, String> attributes, final Map<String, String> flowFileProps,
-            final Map<String, String> env, final Map<?, ?> sysProps) {
-        @SuppressWarnings("rawtypes")
-        final Map[] maps = new Map[]{attributes, flowFileProps, env, sysProps};
-
-        return new Map<String, String>() {
-            @Override
-            public int size() {
-                int size = 0;
-                for (final Map<?, ?> map : maps) {
-                    size += map.size();
-                }
-                return size;
-            }
-
-            @Override
-            public boolean isEmpty() {
-                for (final Map<?, ?> map : maps) {
-                    if (!map.isEmpty()) {
-                        return false;
-                    }
-                }
-                return true;
-            }
-
-            @Override
-            public boolean containsKey(final Object key) {
-                if (key == null) {
-                    return false;
-                }
-                if (!(key instanceof String)) {
-                    return false;
-                }
-
-                for (final Map<?, ?> map : maps) {
-                    if (map.containsKey(key)) {
-                        return true;
-                    }
-                }
-                return false;
-            }
-
-            @Override
-            public boolean containsValue(final Object value) {
-                for (final Map<?, ?> map : maps) {
-                    if (map.containsValue(value)) {
-                        return true;
-                    }
-                }
-                return false;
-            }
-
-            @Override
-            @SuppressWarnings("rawtypes")
-            public String get(final Object key) {
-                if (key == null) {
-                    throw new IllegalArgumentException("Null Keys are not allowed");
-                }
-                if (!(key instanceof String)) {
-                    return null;
-                }
-
-                for (final Map map : maps) {
-                    final Object val = map.get(key);
-                    if (val != null) {
-                        return String.valueOf(val);
-                    }
-                }
-                return null;
-            }
-
-            @Override
-            public String put(String key, String value) {
-                throw new UnsupportedOperationException();
-            }
-
-            @Override
-            public String remove(final Object key) {
-                throw new UnsupportedOperationException();
-            }
-
-            @Override
-            public void putAll(final Map<? extends String, ? extends String> m) {
-                throw new UnsupportedOperationException();
-            }
-
-            @Override
-            public void clear() {
-                throw new UnsupportedOperationException();
-            }
-
-            @Override
-            @SuppressWarnings({"unchecked", "rawtypes"})
-            public Set<String> keySet() {
-                final Set<String> keySet = new HashSet<>();
-                for (final Map map : maps) {
-                    keySet.addAll(map.keySet());
-                }
-                return keySet;
-            }
-
-            @Override
-            @SuppressWarnings({"unchecked", "rawtypes"})
-            public Collection<String> values() {
-                final Set<String> values = new HashSet<>();
-                for (final Map map : maps) {
-                    values.addAll(map.values());
-                }
-                return values;
-            }
-
-            @Override
-            @SuppressWarnings({"unchecked", "rawtypes"})
-            public Set<java.util.Map.Entry<String, String>> entrySet() {
-                final Set<java.util.Map.Entry<String, String>> entrySet = new HashSet<>();
-                for (final Map map : maps) {
-                    entrySet.addAll(map.entrySet());
-                }
-                return entrySet;
-            }
-
-        };
-    }
-
-    public static Query fromTree(final Tree tree, final String text) {
-        return new Query(text, tree, buildEvaluator(tree));
-    }
-
-    public static Tree compileTree(final String query) throws AttributeExpressionLanguageParsingException {
-        try {
-            final CommonTokenStream lexerTokenStream = createTokenStream(query);
-            final AttributeExpressionParser parser = new AttributeExpressionParser(lexerTokenStream);
-            final Tree ast = (Tree) parser.query().getTree();
-            final Tree tree = ast.getChild(0);
-
-            // ensure that we are able to build the evaluators, so that we validate syntax
-            final Evaluator<?> evaluator = buildEvaluator(tree);
-            verifyMappingEvaluatorReduced(evaluator);
-            return tree;
-        } catch (final AttributeExpressionLanguageParsingException e) {
-            throw e;
-        } catch (final Exception e) {
-            throw new AttributeExpressionLanguageParsingException(e);
-        }
-    }
-
-    public static PreparedQuery prepare(final String query) throws AttributeExpressionLanguageParsingException {
-        if (query == null) {
-            return new EmptyPreparedQuery(null);
-        }
-
-        final List<Range> ranges = extractExpressionRanges(query);
-
-        if (ranges.isEmpty()) {
-            return new EmptyPreparedQuery(query.replace("$$", "$"));
-        }
-
-        try {
-            final List<String> substrings = new ArrayList<>();
-            final Map<String, Tree> trees = new HashMap<>();
-
-            int lastIndex = 0;
-            for (final Range range : ranges) {
-                if (range.getStart() > lastIndex) {
-                    substrings.add(query.substring(lastIndex, range.getStart()).replace("$$", "$"));
-                    lastIndex = range.getEnd() + 1;
-                }
-
-                final String treeText = query.substring(range.getStart(), range.getEnd() + 1).replace("$$", "$");
-                substrings.add(treeText);
-                trees.put(treeText, Query.compileTree(treeText));
-                lastIndex = range.getEnd() + 1;
-            }
-
-            final Range lastRange = ranges.get(ranges.size() - 1);
-            if (lastRange.getEnd() + 1 < query.length()) {
-                final String treeText = query.substring(lastRange.getEnd() + 1).replace("$$", "$");
-                substrings.add(treeText);
-            }
-
-            return new StandardPreparedQuery(substrings, trees);
-        } catch (final AttributeExpressionLanguageParsingException e) {
-            return new InvalidPreparedQuery(query, e.getMessage());
-        }
-    }
-
-    public static Query compile(final String query) throws AttributeExpressionLanguageParsingException {
-        try {
-            final CommonTokenStream lexerTokenStream = createTokenStream(query);
-            final AttributeExpressionParser parser = new AttributeExpressionParser(lexerTokenStream);
-            final Tree ast = (Tree) parser.query().getTree();
-            final Tree tree = ast.getChild(0);
-
-            final Evaluator<?> evaluator = buildEvaluator(tree);
-            verifyMappingEvaluatorReduced(evaluator);
-
-            return new Query(query, tree, evaluator);
-        } catch (final AttributeExpressionLanguageParsingException e) {
-            throw e;
-        } catch (final Exception e) {
-            throw new AttributeExpressionLanguageParsingException(e);
-        }
-    }
-
-    private static void verifyMappingEvaluatorReduced(final Evaluator<?> evaluator) {
-        // if the result type of the evaluator is BOOLEAN, then it will always
-        // be reduced when evaluator.
-        final ResultType resultType = evaluator.getResultType();
-        if (resultType == ResultType.BOOLEAN) {
-            return;
-        }
-
-        final Evaluator<?> rootEvaluator = getRootSubjectEvaluator(evaluator);
-        if (rootEvaluator != null && rootEvaluator instanceof MultiAttributeEvaluator) {
-            final MultiAttributeEvaluator multiAttrEval = (MultiAttributeEvaluator) rootEvaluator;
-            switch (multiAttrEval.getEvaluationType()) {
-                case ALL_ATTRIBUTES:
-                case ALL_MATCHING_ATTRIBUTES:
-                case ALL_DELINEATED_VALUES: {
-                    if (!(evaluator instanceof ReduceEvaluator)) {
-                        throw new AttributeExpressionLanguageParsingException("Cannot evaluate expression because it attempts to reference multiple attributes but does not use a reducing function");
-                    }
-                    break;
-                }
-                default:
-                    throw new AttributeExpressionLanguageParsingException("Cannot evaluate expression because it attempts to reference multiple attributes but does not use a reducing function");
-            }
-        }
-    }
-
-    private static CommonTokenStream createTokenStream(final String expression) throws AttributeExpressionLanguageParsingException {
-        final CharStream input = new ANTLRStringStream(expression);
-        final AttributeExpressionLexer lexer = new AttributeExpressionLexer(input);
-        return new CommonTokenStream(lexer);
-    }
-
-    public ResultType getResultType() {
-        return evaluator.getResultType();
-    }
-
-    QueryResult<?> evaluate() {
-        return evaluate(createExpressionMap(null));
-    }
-
-    QueryResult<?> evaluate(final FlowFile flowFile) {
-        return evaluate(createExpressionMap(flowFile));
-    }
-
-    QueryResult<?> evaluate(final Map<String, String> attributes) {
-        if (evaluated.getAndSet(true)) {
-            throw new IllegalStateException("A Query cannot be evaluated more than once");
-        }
-
-        return evaluator.evaluate(attributes);
-    }
-
-    Tree getTree() {
-        return this.tree;
-    }
-
-    @Override
-    public String toString() {
-        return "Query [" + query + "]";
-    }
-
-    private static StringEvaluator newStringLiteralEvaluator(final String literalValue) {
-        if (literalValue == null || literalValue.length() < 2) {
-            return new StringLiteralEvaluator(literalValue);
-        }
-
-        final List<Range> ranges = extractExpressionRanges(literalValue);
-        if (ranges.isEmpty()) {
-            return new StringLiteralEvaluator(literalValue);
-        }
-
-        final List<Evaluator<?>> evaluators = new ArrayList<>();
-
-        int lastIndex = 0;
-        for (final Range range : ranges) {
-            if (range.getStart() > lastIndex) {
-                evaluators.add(newStringLiteralEvaluator(literalValue.substring(lastIndex, range.getStart())));
-            }
-
-            final String treeText = literalValue.substring(range.getStart(), range.getEnd() + 1);
-            evaluators.add(buildEvaluator(compileTree(treeText)));
-            lastIndex = range.getEnd() + 1;
-        }
-
-        final Range lastRange = ranges.get(ranges.size() - 1);
-        if (lastRange.getEnd() + 1 < literalValue.length()) {
-            final String treeText = literalValue.substring(lastRange.getEnd() + 1);
-            evaluators.add(newStringLiteralEvaluator(treeText));
-        }
-
-        if (evaluators.size() == 1) {
-            return toStringEvaluator(evaluators.get(0));
-        }
-
-        StringEvaluator lastEvaluator = toStringEvaluator(evaluators.get(0));
-        for (int i = 1; i < evaluators.size(); i++) {
-            lastEvaluator = new AppendEvaluator(lastEvaluator, toStringEvaluator(evaluators.get(i)));
-        }
-
-        return lastEvaluator;
-    }
-
-    private static Evaluator<?> buildEvaluator(final Tree tree) {
-        switch (tree.getType()) {
-            case EXPRESSION: {
-                return buildExpressionEvaluator(tree);
-            }
-            case ATTRIBUTE_REFERENCE: {
-                final Evaluator<?> childEvaluator = buildEvaluator(tree.getChild(0));
-                if (childEvaluator instanceof MultiAttributeEvaluator) {
-                    return childEvaluator;
-                }
-                return new AttributeEvaluator(toStringEvaluator(childEvaluator));
-            }
-            case MULTI_ATTRIBUTE_REFERENCE: {
-
-                final Tree functionTypeTree = tree.getChild(0);
-                final int multiAttrType = functionTypeTree.getType();
-                if (multiAttrType == ANY_DELINEATED_VALUE || multiAttrType == ALL_DELINEATED_VALUES) {
-                    final StringEvaluator delineatedValueEvaluator = toStringEvaluator(buildEvaluator(tree.getChild(1)));
-                    final StringEvaluator delimiterEvaluator = toStringEvaluator(buildEvaluator(tree.getChild(2)));
-
-                    return new DelineatedAttributeEvaluator(delineatedValueEvaluator, delimiterEvaluator, multiAttrType);
-                }
-
-                final List<String> attributeNames = new ArrayList<>();
-                for (int i = 1; i < tree.getChildCount(); i++) {  // skip the first child because that's the name of the multi-attribute function
-                    attributeNames.add(newStringLiteralEvaluator(tree.getChild(i).getText()).evaluate(null).getValue());
-                }
-
-                switch (multiAttrType) {
-                    case ALL_ATTRIBUTES:
-                        for (final String attributeName : attributeNames) {
-                            try {
-                                FlowFile.KeyValidator.validateKey(attributeName);
-                            } catch (final IllegalArgumentException iae) {
-                                throw new AttributeExpressionLanguageParsingException("Invalid Attribute Name: " + attributeName + ". " + iae.getMessage());
-                            }
-                        }
-
-                        return new MultiNamedAttributeEvaluator(attributeNames, ALL_ATTRIBUTES);
-                    case ALL_MATCHING_ATTRIBUTES:
-                        return new MultiMatchAttributeEvaluator(attributeNames, ALL_MATCHING_ATTRIBUTES);
-                    case ANY_ATTRIBUTE:
-                        for (final String attributeName : attributeNames) {
-                            try {
-                                FlowFile.KeyValidator.validateKey(attributeName);
-                            } catch (final IllegalArgumentException iae) {
-                                throw new AttributeExpressionLanguageParsingException("Invalid Attribute Name: " + attributeName + ". " + iae.getMessage());
-                            }
-                        }
-
-                        return new MultiNamedAttributeEvaluator(attributeNames, ANY_ATTRIBUTE);
-                    case ANY_MATCHING_ATTRIBUTE:
-                        return new MultiMatchAttributeEvaluator(attributeNames, ANY_MATCHING_ATTRIBUTE);
-                    default:
-                        throw new AssertionError("Illegal Multi-Attribute Reference: " + functionTypeTree.toString());
-                }
-            }
-            case ATTR_NAME: {
-                return newStringLiteralEvaluator(tree.getChild(0).getText());
-            }
-            case NUMBER: {
-                return new NumberLiteralEvaluator(tree.getText());
-            }
-            case STRING_LITERAL: {
-                return newStringLiteralEvaluator(tree.getText());
-            }
-            case TRUE:
-            case FALSE:
-                return buildBooleanEvaluator(tree);
-            case UUID: {
-                return new UuidEvaluator();
-            }
-            case NOW: {
-                return new NowEvaluator();
-            }
-            case IP: {
-                try {
-                    return new IPEvaluator();
-                } catch (final UnknownHostException e) {
-                    throw new AttributeExpressionLanguageException(e);
-                }
-            }
-            case HOSTNAME: {
-                if (tree.getChildCount() == 0) {
-                    try {
-                        return new HostnameEvaluator(false);
-                    } catch (UnknownHostException e) {
-                        throw new AttributeExpressionLanguageException(e);
-                    }
-                } else if (tree.getChildCount() == 1) {
-                    final Tree childTree = tree.getChild(0);
-                    try {
-                        switch (childTree.getType()) {
-                            case TRUE:
-                                return new HostnameEvaluator(true);
-                            case FALSE:
-                                return new HostnameEvaluator(false);
-                            default:
-                                throw new AttributeExpressionLanguageParsingException("Call to hostname() must take 0 or 1 (boolean) parameter");
-                        }
-                    } catch (UnknownHostException e) {
-                        throw new AttributeExpressionLanguageException(e);
-                    }
-                } else {
-                    throw new AttributeExpressionLanguageParsingException("Call to hostname() must take 0 or 1 (boolean) parameter");
-                }
-            }
-            case NEXT_INT: {
-                return new OneUpSequenceEvaluator();
-            }
-            default:
-                throw new AttributeExpressionLanguageParsingException("Unexpected token: " + tree.toString());
-        }
-    }
-
-    private static Evaluator<Boolean> buildBooleanEvaluator(final Tree tree) {
-        switch (tree.getType()) {
-            case TRUE:
-                return new BooleanLiteralEvaluator(true);
-            case FALSE:
-                return new BooleanLiteralEvaluator(false);
-        }
-        throw new AttributeExpressionLanguageParsingException("Cannot build Boolean evaluator from tree " + tree.toString());
-    }
-
-    private static Evaluator<?> buildExpressionEvaluator(final Tree tree) {
-        if (tree.getChildCount() == 0) {
-            throw new AttributeExpressionLanguageParsingException("EXPRESSION tree node has no children");
-        }
-
-        final Evaluator<?> evaluator;
-        if (tree.getChildCount() == 1) {
-            evaluator = buildEvaluator(tree.getChild(0));
-        } else {
-            // we can chain together functions in the form of:
-            // ${x:trim():substring(1,2):trim()}
-            // in this case, the subject of the right-most function is the function to its left; its
-            // subject is the function to its left (the first trim()), and its subject is the value of
-            // the 'x' attribute. We accomplish this logic by iterating over all of the children of the
-            // tree from the right-most child going left-ward.
-            evaluator = buildFunctionExpressionEvaluator(tree, 0);
-        }
-
-        Evaluator<?> chosenEvaluator = evaluator;
-        final Evaluator<?> rootEvaluator = getRootSubjectEvaluator(evaluator);
-        if (rootEvaluator != null) {
-            if (rootEvaluator instanceof MultiAttributeEvaluator) {
-                final MultiAttributeEvaluator multiAttrEval = (MultiAttributeEvaluator) rootEvaluator;
-
-                switch (multiAttrEval.getEvaluationType()) {
-                    case ANY_ATTRIBUTE:
-                    case ANY_MATCHING_ATTRIBUTE:
-                    case ANY_DELINEATED_VALUE:
-                        chosenEvaluator = new AnyAttributeEvaluator((BooleanEvaluator) evaluator, multiAttrEval);
-                        break;
-                    case ALL_ATTRIBUTES:
-                    case ALL_MATCHING_ATTRIBUTES:
-                    case ALL_DELINEATED_VALUES: {
-                        final ResultType resultType = evaluator.getResultType();
-                        if (resultType == ResultType.BOOLEAN) {
-                            chosenEvaluator = new AllAttributesEvaluator((BooleanEvaluator) evaluator, multiAttrEval);
-                        } else if (evaluator instanceof ReduceEvaluator) {
-                            chosenEvaluator = new MappingEvaluator((ReduceEvaluator) evaluator, multiAttrEval);
-                        } else {
-                            throw new AttributeExpressionLanguageException("Cannot evaluate Expression because it attempts to reference multiple attributes but does not use a reducing function");
-                        }
-                        break;
-                    }
-                }
-            }
-        }
-
-        return chosenEvaluator;
-    }
-
-    private static Evaluator<?> buildFunctionExpressionEvaluator(final Tree tree, final int offset) {
-        if (tree.getChildCount() == 0) {
-            throw new AttributeExpressionLanguageParsingException("EXPRESSION tree node has no children");
-        }
-        final int firstChildIndex = tree.getChildCount() - offset - 1;
-        if (firstChildIndex == 0) {
-            return buildEvaluator(tree.getChild(0));
-        }
-
-        final Tree functionTree = tree.getChild(firstChildIndex);
-        final Evaluator<?> subjectEvaluator = buildFunctionExpressionEvaluator(tree, offset + 1);
-
-        final Tree functionNameTree = functionTree.getChild(0);
-        final List<Evaluator<?>> argEvaluators = new ArrayList<>();
-        for (int i = 1; i < functionTree.getChildCount(); i++) {
-            argEvaluators.add(buildEvaluator(functionTree.getChild(i)));
-        }
-        return buildFunctionEvaluator(functionNameTree, subjectEvaluator, argEvaluators);
-    }
-
-    private static List<Evaluator<?>> verifyArgCount(final List<Evaluator<?>> args, final int count, final String functionName) {
-        if (args.size() != count) {
-            throw new AttributeExpressionLanguageParsingException(functionName + "() function takes " + count + " arguments");
-        }
-        return args;
-    }
-
-    private static StringEvaluator toStringEvaluator(final Evaluator<?> evaluator) {
-        return toStringEvaluator(evaluator, null);
-    }
-
-    private static StringEvaluator toStringEvaluator(final Evaluator<?> evaluator, final String location) {
-        if (evaluator.getResultType() == ResultType.STRING) {
-            return (StringEvaluator) evaluator;
-        }
-
-        return new StringCastEvaluator(evaluator);
-    }
-
-    private static BooleanEvaluator toBooleanEvaluator(final Evaluator<?> evaluator, final String location) {
-        switch (evaluator.getResultType()) {
-            case BOOLEAN:
-                return (BooleanEvaluator) evaluator;
-            case STRING:
-                return new BooleanCastEvaluator((StringEvaluator) evaluator);
-            default:
-                throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.BOOLEAN
-                        + (location == null ? "" : " at location [" + location + "]"));
-        }
-
-    }
-
-    private static BooleanEvaluator toBooleanEvaluator(final Evaluator<?> evaluator) {
-        return toBooleanEvaluator(evaluator, null);
-    }
-
-    private static NumberEvaluator toNumberEvaluator(final Evaluator<?> evaluator) {
-        return toNumberEvaluator(evaluator, null);
-    }
-
-    private static NumberEvaluator toNumberEvaluator(final Evaluator<?> evaluator, final String location) {
-        switch (evaluator.getResultType()) {
-            case NUMBER:
-                return (NumberEvaluator) evaluator;
-            case STRING:
-                return new NumberCastEvaluator((StringEvaluator) evaluator);
-            case DATE:
-                return new DateToNumberEvaluator((DateEvaluator) evaluator);
-            default:
-                throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.NUMBER
-                        + (location == null ? "" : " at location [" + location + "]"));
-        }
-    }
-
-    private static DateEvaluator toDateEvaluator(final Evaluator<?> evaluator) {
-        return toDateEvaluator(evaluator, null);
-    }
-
-    private static DateEvaluator toDateEvaluator(final Evaluator<?> evaluator, final String location) {
-        if (evaluator.getResultType() == ResultType.DATE) {
-            return (DateEvaluator) evaluator;
-        }
-
-        return new DateCastEvaluator(evaluator);
-    }
-
-    private static Evaluator<?> buildFunctionEvaluator(final Tree tree, final Evaluator<?> subjectEvaluator, final List<Evaluator<?>> argEvaluators) {
-        switch (tree.getType()) {
-            case TRIM: {
-                verifyArgCount(argEvaluators, 0, "trim");
-                return new TrimEvaluator(toStringEvaluator(subjectEvaluator));
-            }
-            case TO_STRING: {
-                verifyArgCount(argEvaluators, 0, "toString");
-                return new ToStringEvaluator(subjectEvaluator);
-            }
-            case TO_LOWER: {
-                verifyArgCount(argEvaluators, 0, "toLower");
-                return new ToLowerEvaluator(toStringEvaluator(subjectEvaluator));
-            }
-            case TO_UPPER: {
-                verifyArgCount(argEvaluators, 0, "toUpper");
-                return new ToUpperEvaluator(toStringEvaluator(subjectEvaluator));
-            }
-            case URL_ENCODE: {
-                verifyArgCount(argEvaluators, 0, "urlEncode");
-                return new UrlEncodeEvaluator(toStringEvaluator(subjectEvaluator));
-            }
-            case URL_DECODE: {
-                verifyArgCount(argEvaluators, 0, "urlDecode");
-                return new UrlDecodeEvaluator(toStringEvaluator(subjectEvaluator));
-            }
-            case SUBSTRING_BEFORE: {
-                verifyArgCount(argEvaluators, 1, "substringBefore");
-                return new SubstringBeforeEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to substringBefore"));
-            }
-            case SUBSTRING_BEFORE_LAST: {
-                verifyArgCount(argEvaluators, 1, "substringBeforeLast");
-                return new SubstringBeforeLastEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to substringBeforeLast"));
-            }
-            case SUBSTRING_AFTER: {
-                verifyArgCount(argEvaluators, 1, "substringAfter");
-                return new SubstringAfterEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to substringAfter"));
-            }
-            case SUBSTRING_AFTER_LAST: {
-                verifyArgCount(argEvaluators, 1, "substringAfterLast");
-                return new SubstringAfterLastEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to substringAfterLast"));
-            }
-            case REPLACE_NULL: {
-                verifyArgCount(argEvaluators, 1, "replaceNull");
-                return new ReplaceNullEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to replaceNull"));
-            }
-            case REPLACE_EMPTY: {
-                verifyArgCount(argEvaluators, 1, "replaceEmtpy");
-                return new ReplaceEmptyEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0), "first argumen to replaceEmpty"));
-            }
-            case REPLACE: {
-                verifyArgCount(argEvaluators, 2, "replace");
-                return new ReplaceEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to replace"),
-                        toStringEvaluator(argEvaluators.get(1), "second argument to replace"));
-            }
-            case REPLACE_ALL: {
-                verifyArgCount(argEvaluators, 2, "replaceAll");
-                return new ReplaceAllEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to replaceAll"),
-                        toStringEvaluator(argEvaluators.get(1), "second argument to replaceAll"));
-            }
-            case APPEND: {
-                verifyArgCount(argEvaluators, 1, "append");
-                return new AppendEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to append"));
-            }
-            case PREPEND: {
-                verifyArgCount(argEvaluators, 1, "prepend");
-                return new PrependEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to prepend"));
-            }
-            case SUBSTRING: {
-                final int numArgs = argEvaluators.size();
-                if (numArgs == 1) {
-                    return new SubstringEvaluator(toStringEvaluator(subjectEvaluator),
-                            toNumberEvaluator(argEvaluators.get(0), "first argument to substring"));
-                } else if (numArgs == 2) {
-                    return new SubstringEvaluator(toStringEvaluator(subjectEvaluator),
-                            toNumberEvaluator(argEvaluators.get(0), "first argument to substring"),
-                            toNumberEvaluator(argEvaluators.get(1), "second argument to substring"));
-                } else {
-                    throw new AttributeExpressionLanguageParsingException("substring() function can take either 1 or 2 arguments but cannot take " + numArgs + " arguments");
-                }
-            }
-            case JOIN: {
-                verifyArgCount(argEvaluators, 1, "join");
-                return new JoinEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)));
-            }
-            case COUNT: {
-                verifyArgCount(argEvaluators, 0, "count");
-                return new CountEvaluator(subjectEvaluator);
-            }
-            case IS_NULL: {
-                verifyArgCount(argEvaluators, 0, "isNull");
-                return new IsNullEvaluator(toStringEvaluator(subjectEvaluator));
-            }
-            case IS_EMPTY: {
-                verifyArgCount(argEvaluators, 0, "isNull");
-                return new IsEmptyEvaluator(toStringEvaluator(subjectEvaluator));
-            }
-            case NOT_NULL: {
-                verifyArgCount(argEvaluators, 0, "notNull");
-                return new NotNullEvaluator(toStringEvaluator(subjectEvaluator));
-            }
-            case STARTS_WITH: {
-                verifyArgCount(argEvaluators, 1, "startsWith");
-                return new StartsWithEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to startsWith"));
-            }
-            case ENDS_WITH: {
-                verifyArgCount(argEvaluators, 1, "endsWith");
-                return new EndsWithEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to endsWith"));
-            }
-            case CONTAINS: {
-                verifyArgCount(argEvaluators, 1, "contains");
-                return new ContainsEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to contains"));
-            }
-            case FIND: {
-                verifyArgCount(argEvaluators, 1, "find");
-                return new FindEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to find"));
-            }
-            case MATCHES: {
-                verifyArgCount(argEvaluators, 1, "matches");
-                return new MatchesEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to matches"));
-            }
-            case EQUALS: {
-                verifyArgCount(argEvaluators, 1, "equals");
-                return new EqualsEvaluator(subjectEvaluator, argEvaluators.get(0));
-            }
-            case EQUALS_IGNORE_CASE: {
-                verifyArgCount(argEvaluators, 1, "equalsIgnoreCase");
-                return new EqualsIgnoreCaseEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to equalsIgnoreCase"));
-            }
-            case GREATER_THAN: {
-                verifyArgCount(argEvaluators, 1, "gt");
-                return new GreaterThanEvaluator(toNumberEvaluator(subjectEvaluator),
-                        toNumberEvaluator(argEvaluators.get(0), "first argument to gt"));
-            }
-            case GREATER_THAN_OR_EQUAL: {
-                verifyArgCount(argEvaluators, 1, "ge");
-                return new GreaterThanOrEqualEvaluator(toNumberEvaluator(subjectEvaluator),
-                        toNumberEvaluator(argEvaluators.get(0), "first argument to ge"));
-            }
-            case LESS_THAN: {
-                verifyArgCount(argEvaluators, 1, "lt");
-                return new LessThanEvaluator(toNumberEvaluator(subjectEvaluator),
-                        toNumberEvaluator(argEvaluators.get(0), "first argument to lt"));
-            }
-            case LESS_THAN_OR_EQUAL: {
-                verifyArgCount(argEvaluators, 1, "le");
-                return new LessThanOrEqualEvaluator(toNumberEvaluator(subjectEvaluator),
-                        toNumberEvaluator(argEvaluators.get(0), "first argument to le"));
-            }
-            case LENGTH: {
-                verifyArgCount(argEvaluators, 0, "length");
-                return new LengthEvaluator(toStringEvaluator(subjectEvaluator));
-            }
-            case TO_DATE: {
-                if (argEvaluators.isEmpty()) {
-                    return new NumberToDateEvaluator(toNumberEvaluator(subjectEvaluator));
-                } else if (subjectEvaluator.getResultType() == ResultType.STRING) {
-                    return new StringToDateEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)));
-                } else {
-                    return new NumberToDateEvaluator(toNumberEvaluator(subjectEvaluator));
-                }
-            }
-            case TO_NUMBER: {
-                verifyArgCount(argEvaluators, 0, "toNumber");
-                switch (subjectEvaluator.getResultType()) {
-                    case STRING:
-                        return new ToNumberEvaluator((StringEvaluator) subjectEvaluator);
-                    case DATE:
-                        return new DateToNumberEvaluator((DateEvaluator) subjectEvaluator);
-                    default:
-                        throw new AttributeExpressionLanguageParsingException(subjectEvaluator + " returns type " + subjectEvaluator.getResultType() + " but expected to get " + ResultType.STRING);
-                }
-            }
-            case TO_RADIX: {
-                if (argEvaluators.size() == 1) {
-                    return new ToRadixEvaluator((NumberEvaluator) subjectEvaluator, toNumberEvaluator(argEvaluators.get(0)));
-                } else {
-                    return new ToRadixEvaluator((NumberEvaluator) subjectEvaluator, toNumberEvaluator(argEvaluators.get(0)), toNumberEvaluator(argEvaluators.get(1)));
-                }
-            }
-            case MOD: {
-                return new ModEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0)));
-            }
-            case PLUS: {
-                return new PlusEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0)));
-            }
-            case MINUS: {
-                return new MinusEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0)));
-            }
-            case MULTIPLY: {
-                return new MultiplyEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0)));
-            }
-            case DIVIDE: {
-                return new DivideEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0)));
-            }
-            case INDEX_OF: {
-                verifyArgCount(argEvaluators, 1, "indexOf");
-                return new IndexOfEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to indexOf"));
-            }
-            case LAST_INDEX_OF: {
-                verifyArgCount(argEvaluators, 1, "lastIndexOf");
-                return new LastIndexOfEvaluator(toStringEvaluator(subjectEvaluator),
-                        toStringEvaluator(argEvaluators.get(0), "first argument to lastIndexOf"));
-            }
-            case FORMAT: {
-                return new FormatEvaluator(toDateEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0), "first argument of format"));
-            }
-            case OR: {
-                return new OrEvaluator(toBooleanEvaluator(subjectEvaluator), toBooleanEvaluator(argEvaluators.get(0)));
-            }
-            case AND: {
-                return new AndEvaluator(toBooleanEvaluator(subjectEvaluator), toBooleanEvaluator(argEvaluators.get(0)));
-            }
-            case NOT: {
-                return new NotEvaluator(toBooleanEvaluator(subjectEvaluator));
-            }
-            default:
-                throw new AttributeExpressionLanguageParsingException("Expected a Function-type expression but got " + tree.toString());
-        }
-    }
-
-    public static class Range {
-
-        private final int start;
-        private final int end;
-
-        public Range(final int start, final int end) {
-            this.start = start;
-            this.end = end;
-        }
-
-        public int getStart() {
-            return start;
-        }
-
-        public int getEnd() {
-            return end;
-        }
-
-        @Override
-        public String toString() {
-            return start + " - " + end;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardAttributeExpression.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardAttributeExpression.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardAttributeExpression.java
deleted file mode 100644
index 49ef6ef..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardAttributeExpression.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.attribute.expression.language;
-
-import org.apache.nifi.expression.AttributeExpression;
-import org.apache.nifi.expression.AttributeValueDecorator;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.exception.ProcessException;
-
-public class StandardAttributeExpression implements AttributeExpression {
-
-    private final Query query;
-
-    public StandardAttributeExpression(final Query query) {
-        this.query = query;
-    }
-
-    @Override
-    public ResultType getResultType() {
-        return query.getResultType();
-    }
-
-    @Override
-    public String evaluate() throws ProcessException {
-        return evaluate((AttributeValueDecorator) null);
-    }
-
-    @Override
-    public String evaluate(final AttributeValueDecorator decorator) throws ProcessException {
-        return evaluate(null, decorator);
-    }
-
-    @Override
-    public String evaluate(final FlowFile flowFile) throws ProcessException {
-        return evaluate(flowFile, null);
-    }
-
-    @Override
-    public String evaluate(final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException {
-        final Object evaluationResult = query.evaluate(flowFile).getValue();
-        if (evaluationResult == null) {
-            return "";
-        }
-
-        String result = evaluationResult.toString();
-        if (decorator != null) {
-            result = decorator.decorate(result);
-        }
-        return Query.unescape(result);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardExpressionLanguageCompiler.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardExpressionLanguageCompiler.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardExpressionLanguageCompiler.java
deleted file mode 100644
index cec73d1..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardExpressionLanguageCompiler.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.attribute.expression.language;
-
-import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
-import org.apache.nifi.expression.AttributeExpression;
-import org.apache.nifi.expression.ExpressionLanguageCompiler;
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-
-public class StandardExpressionLanguageCompiler implements ExpressionLanguageCompiler {
-
-    @Override
-    public AttributeExpression compile(final String expression) throws IllegalArgumentException {
-        try {
-            return new StandardAttributeExpression(Query.compile(expression));
-        } catch (final AttributeExpressionLanguageParsingException e) {
-            throw new IllegalArgumentException(e.getMessage());
-        }
-    }
-
-    @Override
-    public boolean isValidExpression(final String expression) {
-        return Query.isValidExpression(expression);
-    }
-
-    @Override
-    public String validateExpression(final String expression, final boolean allowSurroundingCharacters) {
-        try {
-            Query.validateExpression(expression, allowSurroundingCharacters);
-            return null;
-        } catch (final AttributeExpressionLanguageParsingException aelpe) {
-            return aelpe.getMessage();
-        }
-    }
-
-    @Override
-    public ResultType getResultType(final String expression) throws IllegalArgumentException {
-        try {
-            return Query.getResultType(expression);
-        } catch (final AttributeExpressionLanguageParsingException e) {
-            throw new IllegalArgumentException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
deleted file mode 100644
index 0affb7f..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.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.attribute.expression.language;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.nifi.expression.AttributeValueDecorator;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.exception.ProcessException;
-
-import org.antlr.runtime.tree.Tree;
-
-public class StandardPreparedQuery implements PreparedQuery {
-
-    private final List<String> queryStrings;
-    private final Map<String, Tree> trees;
-
-    public StandardPreparedQuery(final List<String> queryStrings, final Map<String, Tree> trees) {
-        this.queryStrings = new ArrayList<>(queryStrings);
-        this.trees = new HashMap<>(trees);
-    }
-
-    @Override
-    public String evaluateExpressions(Map<String, String> attributes) throws ProcessException {
-        return evaluateExpressions(attributes, null);
-    }
-
-    @Override
-    public String evaluateExpressions(final Map<String, String> attributes, final AttributeValueDecorator decorator) throws ProcessException {
-        final StringBuilder sb = new StringBuilder();
-        for (final String val : queryStrings) {
-            final Tree tree = trees.get(val);
-            if (tree == null) {
-                sb.append(val);
-            } else {
-                final String evaluated = Query.evaluateExpression(tree, val, attributes, decorator);
-                if (evaluated != null) {
-                    sb.append(evaluated);
-                }
-            }
-        }
-        return sb.toString();
-    }
-
-    @Override
-    public String evaluateExpressions(final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException {
-        final Map<String, String> expressionMap = Query.createExpressionMap(flowFile);
-        return evaluateExpressions(expressionMap, decorator);
-    }
-
-    @Override
-    public String evaluateExpressions() throws ProcessException {
-        return evaluateExpressions((FlowFile) null, null);
-    }
-
-    @Override
-    public String evaluateExpressions(final AttributeValueDecorator decorator) throws ProcessException {
-        return evaluateExpressions((FlowFile) null, decorator);
-    }
-
-    @Override
-    public String evaluateExpressions(final FlowFile flowFile) throws ProcessException {
-        return evaluateExpressions(flowFile, null);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanEvaluator.java
deleted file mode 100644
index 376ddfe..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanEvaluator.java
+++ /dev/null
@@ -1,32 +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.attribute.expression.language.evaluation;
-
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-
-public abstract class BooleanEvaluator implements Evaluator<Boolean> {
-
-    @Override
-    public ResultType getResultType() {
-        return ResultType.BOOLEAN;
-    }
-
-    @Override
-    public int getEvaluationsRemaining() {
-        return 0;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanQueryResult.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanQueryResult.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanQueryResult.java
deleted file mode 100644
index e5ef113..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanQueryResult.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.attribute.expression.language.evaluation;
-
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-
-public class BooleanQueryResult implements QueryResult<Boolean> {
-
-    private final Boolean value;
-
-    public BooleanQueryResult(final Boolean value) {
-        this.value = value;
-    }
-
-    @Override
-    public Boolean getValue() {
-        return value;
-    }
-
-    @Override
-    public ResultType getResultType() {
-        return ResultType.BOOLEAN;
-    }
-
-    @Override
-    public String toString() {
-        return String.valueOf(getValue());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java
deleted file mode 100644
index 7474b60..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java
+++ /dev/null
@@ -1,34 +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.attribute.expression.language.evaluation;
-
-import java.util.Date;
-
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-
-public abstract class DateEvaluator implements Evaluator<Date> {
-
-    @Override
-    public ResultType getResultType() {
-        return ResultType.DATE;
-    }
-
-    @Override
-    public int getEvaluationsRemaining() {
-        return 0;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateQueryResult.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateQueryResult.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateQueryResult.java
deleted file mode 100644
index a77bbe9..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateQueryResult.java
+++ /dev/null
@@ -1,45 +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.attribute.expression.language.evaluation;
-
-import java.util.Date;
-
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-
-public class DateQueryResult implements QueryResult<Date> {
-
-    private final Date date;
-
-    public DateQueryResult(final Date date) {
-        this.date = date;
-    }
-
-    @Override
-    public Date getValue() {
-        return date;
-    }
-
-    @Override
-    public ResultType getResultType() {
-        return ResultType.DATE;
-    }
-
-    @Override
-    public String toString() {
-        return String.valueOf(getValue());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java
deleted file mode 100644
index 6d164df..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java
+++ /dev/null
@@ -1,32 +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.attribute.expression.language.evaluation;
-
-import java.util.Map;
-
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-
-public interface Evaluator<T> {
-
-    QueryResult<T> evaluate(Map<String, String> attributes);
-
-    ResultType getResultType();
-
-    int getEvaluationsRemaining();
-
-    Evaluator<?> getSubjectEvaluator();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java
deleted file mode 100644
index 403bae3..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java
+++ /dev/null
@@ -1,33 +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.attribute.expression.language.evaluation;
-
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-
-public abstract class NumberEvaluator implements Evaluator<Long> {
-
-    @Override
-    public ResultType getResultType() {
-        return ResultType.NUMBER;
-    }
-
-    @Override
-    public int getEvaluationsRemaining() {
-        return 0;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberQueryResult.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberQueryResult.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberQueryResult.java
deleted file mode 100644
index fc3c961..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberQueryResult.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.attribute.expression.language.evaluation;
-
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-
-public class NumberQueryResult implements QueryResult<Long> {
-
-    private final Long value;
-
-    public NumberQueryResult(final Long value) {
-        this.value = value;
-    }
-
-    @Override
-    public Long getValue() {
-        return value;
-    }
-
-    @Override
-    public ResultType getResultType() {
-        return ResultType.NUMBER;
-    }
-
-    @Override
-    public String toString() {
-        return String.valueOf(getValue());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/QueryResult.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/QueryResult.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/QueryResult.java
deleted file mode 100644
index 56bd76a..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/QueryResult.java
+++ /dev/null
@@ -1,26 +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.attribute.expression.language.evaluation;
-
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-
-public interface QueryResult<T> {
-
-    T getValue();
-
-    ResultType getResultType();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringEvaluator.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringEvaluator.java
deleted file mode 100644
index 1f4ff21..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringEvaluator.java
+++ /dev/null
@@ -1,32 +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.attribute.expression.language.evaluation;
-
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-
-public abstract class StringEvaluator implements Evaluator<String> {
-
-    @Override
-    public ResultType getResultType() {
-        return ResultType.STRING;
-    }
-
-    @Override
-    public int getEvaluationsRemaining() {
-        return 0;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringQueryResult.java
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringQueryResult.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringQueryResult.java
deleted file mode 100644
index c010bb1..0000000
--- a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringQueryResult.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.attribute.expression.language.evaluation;
-
-import org.apache.nifi.expression.AttributeExpression.ResultType;
-
-public class StringQueryResult implements QueryResult<String> {
-
-    private final String value;
-
-    public StringQueryResult(final String value) {
-        this.value = value;
-    }
-
-    @Override
-    public String getValue() {
-        return value;
-    }
-
-    @Override
-    public ResultType getResultType() {
-        return ResultType.STRING;
-    }
-
-    @Override
-    public String toString() {
-        return String.valueOf(getValue());
-    }
-}


[37/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java
deleted file mode 100644
index 41a0557..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java
+++ /dev/null
@@ -1,623 +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.file;
-
-import java.io.BufferedInputStream;
-import java.io.Closeable;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.channels.FileChannel;
-import java.nio.channels.FileLock;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.Random;
-
-import org.slf4j.Logger;
-
-/**
- * A utility class containing a few useful static methods to do typical IO
- * operations.
- *
- * @author unattributed
- */
-public class FileUtils {
-
-    public static final long TRANSFER_CHUNK_SIZE_BYTES = 1024 * 1024 * 8; //8 MB chunks
-    public static final long MILLIS_BETWEEN_ATTEMPTS = 50L;
-
-    /**
-     * Closes the given closeable quietly - no logging, no exceptions...
-     *
-     * @param closeable
-     */
-    public static void closeQuietly(final Closeable closeable) {
-        if (null != closeable) {
-            try {
-                closeable.close();
-            } catch (final IOException io) {/*IGNORE*/
-
-            }
-        }
-    }
-
-    /**
-     * Releases the given lock quietly - no logging, no exception
-     *
-     * @param lock
-     */
-    public static void releaseQuietly(final FileLock lock) {
-        if (null != lock) {
-            try {
-                lock.release();
-            } catch (final IOException io) {
-                /*IGNORE*/
-            }
-        }
-    }
-
-    public static void ensureDirectoryExistAndCanAccess(final File dir) throws IOException {
-        if (dir.exists() && !dir.isDirectory()) {
-            throw new IOException(dir.getAbsolutePath() + " is not a directory");
-        } else if (!dir.exists()) {
-            final boolean made = dir.mkdirs();
-            if (!made) {
-                throw new IOException(dir.getAbsolutePath() + " could not be created");
-            }
-        }
-        if (!(dir.canRead() && dir.canWrite())) {
-            throw new IOException(dir.getAbsolutePath() + " directory does not have read/write privilege");
-        }
-    }
-
-    /**
-     * Deletes the given file. If the given file exists but could not be deleted
-     * this will be printed as a warning to the given logger
-     *
-     * @param file
-     * @param logger
-     * @return
-     */
-    public static boolean deleteFile(final File file, final Logger logger) {
-        return FileUtils.deleteFile(file, logger, 1);
-    }
-
-    /**
-     * Deletes the given file. If the given file exists but could not be deleted
-     * this will be printed as a warning to the given logger
-     *
-     * @param file
-     * @param logger
-     * @param attempts indicates how many times an attempt to delete should be
-     * made
-     * @return true if given file no longer exists
-     */
-    public static boolean deleteFile(final File file, final Logger logger, final int attempts) {
-        if (file == null) {
-            return false;
-        }
-        boolean isGone = false;
-        try {
-            if (file.exists()) {
-                final int effectiveAttempts = Math.max(1, attempts);
-                for (int i = 0; i < effectiveAttempts && !isGone; i++) {
-                    isGone = file.delete() || !file.exists();
-                    if (!isGone && (effectiveAttempts - i) > 1) {
-                        FileUtils.sleepQuietly(MILLIS_BETWEEN_ATTEMPTS);
-                    }
-                }
-                if (!isGone && logger != null) {
-                    logger.warn("File appears to exist but unable to delete file: " + file.getAbsolutePath());
-                }
-            }
-        } catch (final Throwable t) {
-            if (logger != null) {
-                logger.warn("Unable to delete file: '" + file.getAbsolutePath() + "' due to " + t);
-            }
-        }
-        return isGone;
-    }
-
-    /**
-     * Deletes all of the given files. If any exist and cannot be deleted that
-     * will be printed at warn to the given logger.
-     *
-     * @param files can be null
-     * @param logger can be null
-     */
-    public static void deleteFile(final List<File> files, final Logger logger) {
-        FileUtils.deleteFile(files, logger, 1);
-    }
-
-    /**
-     * Deletes all of the given files. If any exist and cannot be deleted that
-     * will be printed at warn to the given logger.
-     *
-     * @param files can be null
-     * @param logger can be null
-     * @param attempts indicates how many times an attempt should be made to
-     * delete each file
-     */
-    public static void deleteFile(final List<File> files, final Logger logger, final int attempts) {
-        if (null == files || files.isEmpty()) {
-            return;
-        }
-        final int effectiveAttempts = Math.max(1, attempts);
-        for (final File file : files) {
-            try {
-                boolean isGone = false;
-                for (int i = 0; i < effectiveAttempts && !isGone; i++) {
-                    isGone = file.delete() || !file.exists();
-                    if (!isGone && (effectiveAttempts - i) > 1) {
-                        FileUtils.sleepQuietly(MILLIS_BETWEEN_ATTEMPTS);
-                    }
-                }
-                if (!isGone && logger != null) {
-                    logger.warn("File appears to exist but unable to delete file: " + file.getAbsolutePath());
-                }
-            } catch (final Throwable t) {
-                if (null != logger) {
-                    logger.warn("Unable to delete file given from path: '" + file.getPath() + "' due to " + t);
-                }
-            }
-        }
-    }
-
-    /**
-     * 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
-     * @param filter if null then no filter is used
-     * @param logger
-     */
-    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
-     * @param filter if null then no filter is used
-     * @param logger
-     * @param recurse
-     */
-    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
-     * @param filter if null then no filter is used
-     * @param logger
-     * @param recurse
-     * @param deleteEmptyDirectories default is false; if true will delete
-     * directories found that are empty
-     */
-    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();
-            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 given files.
-     *
-     * @param files
-     * @param recurse will recurse
-     * @throws IOException
-     */
-    public static void deleteFiles(final Collection<File> files, final boolean recurse) throws IOException {
-        for (final File file : files) {
-            FileUtils.deleteFile(file, recurse);
-        }
-    }
-
-    public static void deleteFile(final File file, final boolean recurse) throws IOException {
-        if (file.isDirectory() && recurse) {
-            FileUtils.deleteFiles(Arrays.asList(file.listFiles()), recurse);
-        }
-        //now delete the file itself regardless of whether it is plain file or a directory
-        if (!FileUtils.deleteFile(file, null, 5)) {
-            throw new IOException("Unable to delete " + file.getAbsolutePath());
-        }
-    }
-
-    /**
-     * Randomly generates a sequence of bytes and overwrites the contents of the
-     * file a number of times. The file is then deleted.
-     *
-     * @param file File to be overwritten a number of times and, ultimately,
-     * deleted
-     * @param passes Number of times file should be overwritten
-     * @throws IOException if something makes shredding or deleting a problem
-     */
-    public static void shredFile(final File file, final int passes)
-            throws IOException {
-        final Random generator = new Random();
-        final long fileLength = file.length();
-        final int byteArraySize = (int) Math.min(fileLength, 1048576); // 1MB
-        final byte[] b = new byte[byteArraySize];
-        final long numOfRandomWrites = (fileLength / b.length) + 1;
-        final FileOutputStream fos = new FileOutputStream(file);
-        try {
-            // Over write file contents (passes) times
-            final FileChannel channel = fos.getChannel();
-            for (int i = 0; i < passes; i++) {
-                generator.nextBytes(b);
-                for (int j = 0; j <= numOfRandomWrites; j++) {
-                    fos.write(b);
-                }
-                fos.flush();
-                channel.position(0);
-            }
-            // Write out "0" for each byte in the file
-            Arrays.fill(b, (byte) 0);
-            for (int j = 0; j < numOfRandomWrites; j++) {
-                fos.write(b);
-            }
-            fos.flush();
-            fos.close();
-            // Try to delete the file a few times
-            if (!FileUtils.deleteFile(file, null, 5)) {
-                throw new IOException("Failed to delete file after shredding");
-            }
-
-        } finally {
-            FileUtils.closeQuietly(fos);
-        }
-    }
-
-    public static long copy(final InputStream in, final OutputStream out) throws IOException {
-        final byte[] buffer = new byte[65536];
-        long copied = 0L;
-        int len;
-        while ((len = in.read(buffer)) > 0) {
-            out.write(buffer, 0, len);
-            copied += len;
-        }
-
-        return copied;
-    }
-
-    public static long copyBytes(final byte[] bytes, final File destination, final boolean lockOutputFile) throws FileNotFoundException, IOException {
-        FileOutputStream fos = null;
-        FileLock outLock = null;
-        long fileSize = 0L;
-        try {
-            fos = new FileOutputStream(destination);
-            final FileChannel out = fos.getChannel();
-            if (lockOutputFile) {
-                outLock = out.tryLock(0, Long.MAX_VALUE, false);
-                if (null == outLock) {
-                    throw new IOException("Unable to obtain exclusive file lock for: " + destination.getAbsolutePath());
-                }
-            }
-            fos.write(bytes);
-            fos.flush();
-            fileSize = bytes.length;
-        } finally {
-            FileUtils.releaseQuietly(outLock);
-            FileUtils.closeQuietly(fos);
-        }
-        return fileSize;
-    }
-
-    /**
-     * Copies the given source file to the given destination file. The given
-     * destination will be overwritten if it already exists.
-     *
-     * @param source
-     * @param destination
-     * @param lockInputFile if true will lock input file during copy; if false
-     * will not
-     * @param lockOutputFile if true will lock output file during copy; if false
-     * will not
-     * @param move if true will perform what is effectively a move operation
-     * rather than a pure copy. This allows for potentially highly efficient
-     * movement of the file but if not possible this will revert to a copy then
-     * delete behavior. If false, then the file is copied and the source file is
-     * retained. If a true rename/move occurs then no lock is held during that
-     * time.
-     * @param logger if failures occur, they will be logged to this logger if
-     * possible. If this logger is null, an IOException will instead be thrown,
-     * indicating the problem.
-     * @return long number of bytes copied
-     * @throws FileNotFoundException if the source file could not be found
-     * @throws IOException
-     * @throws SecurityException if a security manager denies the needed file
-     * operations
-     */
-    public static long copyFile(final File source, final File destination, final boolean lockInputFile, final boolean lockOutputFile, final boolean move, final Logger logger) throws FileNotFoundException, IOException {
-
-        FileInputStream fis = null;
-        FileOutputStream fos = null;
-        FileLock inLock = null;
-        FileLock outLock = null;
-        long fileSize = 0L;
-        if (!source.canRead()) {
-            throw new IOException("Must at least have read permission");
-
-        }
-        if (move && source.renameTo(destination)) {
-            fileSize = destination.length();
-        } else {
-            try {
-                fis = new FileInputStream(source);
-                fos = new FileOutputStream(destination);
-                final FileChannel in = fis.getChannel();
-                final FileChannel out = fos.getChannel();
-                if (lockInputFile) {
-                    inLock = in.tryLock(0, Long.MAX_VALUE, true);
-                    if (null == inLock) {
-                        throw new IOException("Unable to obtain shared file lock for: " + source.getAbsolutePath());
-                    }
-                }
-                if (lockOutputFile) {
-                    outLock = out.tryLock(0, Long.MAX_VALUE, false);
-                    if (null == outLock) {
-                        throw new IOException("Unable to obtain exclusive file lock for: " + destination.getAbsolutePath());
-                    }
-                }
-                long bytesWritten = 0;
-                do {
-                    bytesWritten += out.transferFrom(in, bytesWritten, TRANSFER_CHUNK_SIZE_BYTES);
-                    fileSize = in.size();
-                } while (bytesWritten < fileSize);
-                out.force(false);
-                FileUtils.closeQuietly(fos);
-                FileUtils.closeQuietly(fis);
-                fos = null;
-                fis = null;
-                if (move && !FileUtils.deleteFile(source, null, 5)) {
-                    if (logger == null) {
-                        FileUtils.deleteFile(destination, null, 5);
-                        throw new IOException("Could not remove file " + source.getAbsolutePath());
-                    } else {
-                        logger.warn("Configured to delete source file when renaming/move not successful.  However, unable to delete file at: " + source.getAbsolutePath());
-                    }
-                }
-            } finally {
-                FileUtils.releaseQuietly(inLock);
-                FileUtils.releaseQuietly(outLock);
-                FileUtils.closeQuietly(fos);
-                FileUtils.closeQuietly(fis);
-            }
-        }
-        return fileSize;
-    }
-
-    /**
-     * Copies the given source file to the given destination file. The given
-     * destination will be overwritten if it already exists.
-     *
-     * @param source
-     * @param destination
-     * @param lockInputFile if true will lock input file during copy; if false
-     * will not
-     * @param lockOutputFile if true will lock output file during copy; if false
-     * will not
-     * @param logger
-     * @return long number of bytes copied
-     * @throws FileNotFoundException if the source file could not be found
-     * @throws IOException
-     * @throws SecurityException if a security manager denies the needed file
-     * operations
-     */
-    public static long copyFile(final File source, final File destination, final boolean lockInputFile, final boolean lockOutputFile, final Logger logger) throws FileNotFoundException, IOException {
-        return FileUtils.copyFile(source, destination, lockInputFile, lockOutputFile, false, logger);
-    }
-
-    public static long copyFile(final File source, final OutputStream stream, final boolean closeOutputStream, final boolean lockInputFile) throws FileNotFoundException, IOException {
-        FileInputStream fis = null;
-        FileLock inLock = null;
-        long fileSize = 0L;
-        try {
-            fis = new FileInputStream(source);
-            final FileChannel in = fis.getChannel();
-            if (lockInputFile) {
-                inLock = in.tryLock(0, Long.MAX_VALUE, true);
-                if (inLock == null) {
-                    throw new IOException("Unable to obtain exclusive file lock for: " + source.getAbsolutePath());
-                }
-
-            }
-
-            byte[] buffer = new byte[1 << 18]; //256 KB
-            int bytesRead = -1;
-            while ((bytesRead = fis.read(buffer)) != -1) {
-                stream.write(buffer, 0, bytesRead);
-            }
-            in.force(false);
-            stream.flush();
-            fileSize = in.size();
-        } finally {
-            FileUtils.releaseQuietly(inLock);
-            FileUtils.closeQuietly(fis);
-            if (closeOutputStream) {
-                FileUtils.closeQuietly(stream);
-            }
-        }
-        return fileSize;
-    }
-
-    public static long copyFile(final InputStream stream, final File destination, final boolean closeInputStream, final boolean lockOutputFile) throws FileNotFoundException, IOException {
-        final Path destPath = destination.toPath();
-        final long size = Files.copy(stream, destPath);
-        if (closeInputStream) {
-            stream.close();
-        }
-        return size;
-    }
-
-    /**
-     * Renames the given file from the source path to the destination path. This
-     * handles multiple attempts. This should only be used to rename within a
-     * given directory. Renaming across directories might not work well. See the
-     * <code>File.renameTo</code> for more information.
-     *
-     * @param source the file to rename
-     * @param destination the file path to rename to
-     * @param maxAttempts the max number of attempts to attempt the rename
-     * @throws IOException if rename isn't successful
-     */
-    public static void renameFile(final File source, final File destination, final int maxAttempts) throws IOException {
-        FileUtils.renameFile(source, destination, maxAttempts, false);
-    }
-
-    /**
-     * Renames the given file from the source path to the destination path. This
-     * handles multiple attempts. This should only be used to rename within a
-     * given directory. Renaming across directories might not work well. See the
-     * <code>File.renameTo</code> for more information.
-     *
-     * @param source the file to rename
-     * @param destination the file path to rename to
-     * @param maxAttempts the max number of attempts to attempt the rename
-     * @param replace if true and a rename attempt fails will check if a file is
-     * already at the destination path. If so it will delete that file and
-     * attempt the rename according the remaining maxAttempts. If false, any
-     * conflicting files will be left as they were and the rename attempts will
-     * fail if conflicting.
-     * @throws IOException if rename isn't successful
-     */
-    public static void renameFile(final File source, final File destination, final int maxAttempts, final boolean replace) throws IOException {
-        final int attempts = (replace || maxAttempts < 1) ? Math.max(2, maxAttempts) : maxAttempts;
-        boolean renamed = false;
-        for (int i = 0; i < attempts; i++) {
-            renamed = source.renameTo(destination);
-            if (!renamed) {
-                FileUtils.deleteFile(destination, null, 5);
-            } else {
-                break; //rename has succeeded
-            }
-        }
-        if (!renamed) {
-            throw new IOException("Attempted " + maxAttempts + " times but unable to rename from \'" + source.getPath() + "\' to \'" + destination.getPath() + "\'");
-
-        }
-    }
-
-    public static void sleepQuietly(final long millis) {
-        try {
-            Thread.sleep(millis);
-        } catch (final InterruptedException ex) {
-            /* do nothing */
-        }
-    }
-
-    /**
-     * Syncs a primary copy of a file with the copy in the restore directory. If
-     * the restore directory does not have a file and the primary has a file,
-     * the the primary's file is copied to the restore directory. Else if the
-     * restore directory has a file, but the primary does not, then the
-     * restore's file is copied to the primary directory. Else if the primary
-     * file is different than the restore file, then an IllegalStateException is
-     * thrown. Otherwise, if neither file exists, then no syncing is performed.
-     *
-     * @param primaryFile the primary file
-     * @param restoreFile the restore file
-     * @param logger a logger
-     * @throws IOException if an I/O problem was encountered during syncing
-     * @throws IllegalStateException if the primary and restore copies exist but
-     * are different
-     */
-    public static void syncWithRestore(final File primaryFile, final File restoreFile, final Logger logger)
-            throws IOException {
-
-        if (primaryFile.exists() && !restoreFile.exists()) {
-            // copy primary file to restore
-            copyFile(primaryFile, restoreFile, false, false, logger);
-        } else if (restoreFile.exists() && !primaryFile.exists()) {
-            // copy restore file to primary
-            copyFile(restoreFile, primaryFile, false, false, logger);
-        } else if (primaryFile.exists() && restoreFile.exists() && !isSame(primaryFile, restoreFile)) {
-            throw new IllegalStateException(String.format("Primary file '%s' is different than restore file '%s'",
-                    primaryFile.getAbsoluteFile(), restoreFile.getAbsolutePath()));
-        }
-    }
-
-    /**
-     * Returns true if the given files are the same according to their MD5 hash.
-     *
-     * @param file1 a file
-     * @param file2 a file
-     * @return true if the files are the same; false otherwise
-     * @throws IOException if the MD5 hash could not be computed
-     */
-    public static boolean isSame(final File file1, final File file2) throws IOException {
-        return Arrays.equals(computeMd5Digest(file1), computeMd5Digest(file2));
-    }
-
-    /**
-     * Returns the MD5 hash of the given file.
-     *
-     * @param file a file
-     * @return the MD5 hash
-     * @throws IOException if the MD5 hash could not be computed
-     */
-    public static byte[] computeMd5Digest(final File file) throws IOException {
-        final MessageDigest digest;
-        try {
-            digest = MessageDigest.getInstance("MD5");
-        } catch (final NoSuchAlgorithmException nsae) {
-            throw new IOException(nsae);
-        }
-
-        try (final FileInputStream fis = new FileInputStream(file)) {
-            int len;
-            final byte[] buffer = new byte[8192];
-            while ((len = fis.read(buffer)) > -1) {
-                if (len > 0) {
-                    digest.update(buffer, 0, len);
-                }
-            }
-        }
-        return digest.digest();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/CompoundUpdateMonitor.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/CompoundUpdateMonitor.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/CompoundUpdateMonitor.java
deleted file mode 100644
index 6f9c616..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/CompoundUpdateMonitor.java
+++ /dev/null
@@ -1,115 +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.file.monitor;
-
-import java.io.IOException;
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * An {@link UpdateMonitor} that combines multiple <code>UpdateMonitor</code>s
- * such that it will indicate a change in a file only if ALL sub-monitors
- * indicate a change. The sub-monitors will be applied in the order given and if
- * any indicates that the state has not changed, the subsequent sub-monitors may
- * not be given a chance to run
- */
-public class CompoundUpdateMonitor implements UpdateMonitor {
-
-    private final List<UpdateMonitor> monitors;
-
-    public CompoundUpdateMonitor(final UpdateMonitor first, final UpdateMonitor... others) {
-        monitors = new ArrayList<>();
-        monitors.add(first);
-        for (final UpdateMonitor monitor : others) {
-            monitors.add(monitor);
-        }
-    }
-
-    @Override
-    public Object getCurrentState(final Path path) throws IOException {
-        return new DeferredMonitorAction(monitors, path);
-    }
-
-    private static class DeferredMonitorAction {
-
-        private static final Object NON_COMPUTED_VALUE = new Object();
-
-        private final List<UpdateMonitor> monitors;
-        private final Path path;
-
-        private final Object[] preCalculated;
-
-        public DeferredMonitorAction(final List<UpdateMonitor> monitors, final Path path) {
-            this.monitors = monitors;
-            this.path = path;
-            preCalculated = new Object[monitors.size()];
-
-            for (int i = 0; i < preCalculated.length; i++) {
-                preCalculated[i] = NON_COMPUTED_VALUE;
-            }
-        }
-
-        private Object getCalculatedValue(final int i) throws IOException {
-            if (preCalculated[i] == NON_COMPUTED_VALUE) {
-                preCalculated[i] = monitors.get(i).getCurrentState(path);
-            }
-
-            return preCalculated[i];
-        }
-
-        @Override
-        public boolean equals(final Object obj) {
-            // must return true unless ALL DeferredMonitorAction's indicate that they are different
-            if (obj == null) {
-                return false;
-            }
-
-            if (!(obj instanceof DeferredMonitorAction)) {
-                return false;
-            }
-
-            final DeferredMonitorAction other = (DeferredMonitorAction) obj;
-            try {
-                // Go through each UpdateMonitor's value and check if the value has changed.
-                for (int i = 0; i < preCalculated.length; i++) {
-                    final Object mine = getCalculatedValue(i);
-                    final Object theirs = other.getCalculatedValue(i);
-
-                    if (mine == theirs) {
-                        // same
-                        return true;
-                    }
-
-                    if (mine == null && theirs == null) {
-                        // same
-                        return true;
-                    }
-
-                    if (mine.equals(theirs)) {
-                        return true;
-                    }
-                }
-            } catch (final IOException e) {
-                return false;
-            }
-
-            // No DeferredMonitorAction was the same as last time. Therefore, it's not equal
-            return false;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/LastModifiedMonitor.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/LastModifiedMonitor.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/LastModifiedMonitor.java
deleted file mode 100644
index e6be558..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/LastModifiedMonitor.java
+++ /dev/null
@@ -1,30 +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.file.monitor;
-
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-
-public class LastModifiedMonitor implements UpdateMonitor {
-
-    @Override
-    public Object getCurrentState(final Path path) throws IOException {
-        return Files.getLastModifiedTime(path);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/MD5SumMonitor.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/MD5SumMonitor.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/MD5SumMonitor.java
deleted file mode 100644
index 8dea4bf..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/MD5SumMonitor.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.util.file.monitor;
-
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.file.Path;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-
-public class MD5SumMonitor implements UpdateMonitor {
-
-    @Override
-    public Object getCurrentState(final Path path) throws IOException {
-        final MessageDigest digest;
-        try {
-            digest = MessageDigest.getInstance("MD5");
-        } catch (final NoSuchAlgorithmException nsae) {
-            throw new AssertionError(nsae);
-        }
-
-        try (final FileInputStream fis = new FileInputStream(path.toFile())) {
-            int len;
-            final byte[] buffer = new byte[8192];
-            while ((len = fis.read(buffer)) > -1) {
-                if (len > 0) {
-                    digest.update(buffer, 0, len);
-                }
-            }
-        }
-
-        // Return a ByteBuffer instead of byte[] because we want equals() to do a deep equality
-        return ByteBuffer.wrap(digest.digest());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/SynchronousFileWatcher.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/SynchronousFileWatcher.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/SynchronousFileWatcher.java
deleted file mode 100644
index e0089c1..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/SynchronousFileWatcher.java
+++ /dev/null
@@ -1,123 +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.file.monitor;
-
-import java.io.IOException;
-import java.nio.file.Path;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-/**
- * Allows the user to configure a {@link java.nio.file.Path Path} to watch for
- * modifications and periodically poll to check if the file has been modified
- */
-public class SynchronousFileWatcher {
-
-    private final Path path;
-    private final long checkUpdateMillis;
-    private final UpdateMonitor monitor;
-    private final AtomicReference<StateWrapper> lastState;
-    private final Lock resourceLock = new ReentrantLock();
-
-    public SynchronousFileWatcher(final Path path, final UpdateMonitor monitor) {
-        this(path, monitor, 0L);
-    }
-
-    public SynchronousFileWatcher(final Path path, final UpdateMonitor monitor, final long checkMillis) {
-        if (checkMillis < 0) {
-            throw new IllegalArgumentException();
-        }
-
-        this.path = path;
-        checkUpdateMillis = checkMillis;
-        this.monitor = monitor;
-
-        Object currentState;
-        try {
-            currentState = monitor.getCurrentState(path);
-        } catch (final IOException e) {
-            currentState = null;
-        }
-
-        this.lastState = new AtomicReference<>(new StateWrapper(currentState));
-    }
-
-    /**
-     * Checks if the file has been updated according to the configured
-     * {@link UpdateMonitor} and resets the state
-     *
-     * @return
-     * @throws IOException
-     */
-    public boolean checkAndReset() throws IOException {
-        if (checkUpdateMillis <= 0) { // if checkUpdateMillis <= 0, always check
-            return checkForUpdate();
-        } else {
-            final StateWrapper stateWrapper = lastState.get();
-            if (stateWrapper.getTimestamp() < System.currentTimeMillis() - checkUpdateMillis) {
-                return checkForUpdate();
-            }
-            return false;
-        }
-    }
-
-    private boolean checkForUpdate() throws IOException {
-        if (resourceLock.tryLock()) {
-            try {
-                final StateWrapper wrapper = lastState.get();
-                final Object newState = monitor.getCurrentState(path);
-                if (newState == null && wrapper.getState() == null) {
-                    return false;
-                }
-                if (newState == null || wrapper.getState() == null) {
-                    lastState.set(new StateWrapper(newState));
-                    return true;
-                }
-
-                final boolean unmodified = newState.equals(wrapper.getState());
-                if (!unmodified) {
-                    lastState.set(new StateWrapper(newState));
-                }
-                return !unmodified;
-            } finally {
-                resourceLock.unlock();
-            }
-        } else {
-            return false;
-        }
-    }
-
-    private static class StateWrapper {
-
-        private final Object state;
-        private final long timestamp;
-
-        public StateWrapper(final Object state) {
-            this.state = state;
-            this.timestamp = System.currentTimeMillis();
-        }
-
-        public Object getState() {
-            return state;
-        }
-
-        public long getTimestamp() {
-            return timestamp;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/UpdateMonitor.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/UpdateMonitor.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/UpdateMonitor.java
deleted file mode 100644
index 20ed1dd..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/file/monitor/UpdateMonitor.java
+++ /dev/null
@@ -1,25 +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.file.monitor;
-
-import java.io.IOException;
-import java.nio.file.Path;
-
-public interface UpdateMonitor {
-
-    Object getCurrentState(Path path) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/Search.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/Search.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/Search.java
deleted file mode 100644
index 59b444a..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/Search.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.util.search;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Set;
-
-import org.apache.nifi.util.search.ahocorasick.SearchState;
-
-/**
- * Defines an interface to search for content given a set of search terms. Any
- * implementation of search must be thread safe.
- *
- * @author
- * @param <T>
- */
-public interface Search<T> {
-
-    /**
-     * Establishes the dictionary of terms which will be searched in subsequent
-     * search calls. This can be called only once
-     *
-     * @param terms
-     */
-    void initializeDictionary(Set<SearchTerm<T>> terms);
-
-    /**
-     * Searches the given input stream for matches between the already specified
-     * dictionary and the contents scanned.
-     *
-     * @param haystack
-     * @param findAll if true will find all matches if false will find only the
-     * first match
-     * @return SearchState containing results Map might be empty which indicates
-     * no matches found but will not be null
-     * @throws IOException Thrown for any exceptions occurring while searching.
-     * @throws IllegalStateException if the dictionary has not yet been
-     * initialized
-     */
-    SearchState<T> search(InputStream haystack, boolean findAll) throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/SearchTerm.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/SearchTerm.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/SearchTerm.java
deleted file mode 100644
index 62de964..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/SearchTerm.java
+++ /dev/null
@@ -1,141 +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.search;
-
-import java.nio.charset.Charset;
-import java.util.Arrays;
-
-/**
- * This is an immutable thread safe object representing a search term
- *
- * @author
- * @param <T>
- */
-public class SearchTerm<T> {
-
-    private final byte[] bytes;
-    private final int hashCode;
-    private final T reference;
-
-    /**
-     * Constructs a SearchTerm. Defensively copies the given byte array
-     *
-     * @param bytes
-     * @throws IllegalArgument exception if given bytes are null or 0 length
-     */
-    public SearchTerm(final byte[] bytes) {
-        this(bytes, true, null);
-    }
-
-    /**
-     * Constructs a search term. Optionally performs a defensive copy of the
-     * given byte array. If the caller indicates a defensive copy is not
-     * necessary then they must not change the given arrays state any longer
-     *
-     * @param bytes
-     * @param defensiveCopy
-     * @param reference
-     */
-    public SearchTerm(final byte[] bytes, final boolean defensiveCopy, final T reference) {
-        if (bytes == null || bytes.length == 0) {
-            throw new IllegalArgumentException();
-        }
-        if (defensiveCopy) {
-            this.bytes = Arrays.copyOf(bytes, bytes.length);
-        } else {
-            this.bytes = bytes;
-        }
-        this.hashCode = Arrays.hashCode(this.bytes);
-        this.reference = reference;
-    }
-
-    public int get(final int index) {
-        return bytes[index] & 0xff;
-    }
-
-    /**
-     * @return size in of search term in bytes
-     */
-    public int size() {
-        return bytes.length;
-    }
-
-    /**
-     * @return reference object for this given search term
-     */
-    public T getReference() {
-        return reference;
-    }
-
-    /**
-     * Determines if the given window starts with the same bytes as this term
-     *
-     * @param window Current window of bytes from the haystack being evaluated.
-     * @param windowLength The length of the window to consider
-     * @return true if this term starts with the same bytes of the given window
-     */
-    public boolean startsWith(byte[] window, int windowLength) {
-        if (windowLength > window.length) {
-            throw new IndexOutOfBoundsException();
-        }
-        if (bytes.length < windowLength) {
-            return false;
-        }
-        for (int i = 0; i < bytes.length && i < windowLength; i++) {
-            if (bytes[i] != window[i]) {
-                return false;
-            }
-        }
-        return true;
-    }
-
-    /**
-     * @return a defensive copy of the internal byte structure
-     */
-    public byte[] getBytes() {
-        return Arrays.copyOf(bytes, bytes.length);
-    }
-
-    @Override
-    public int hashCode() {
-        return hashCode;
-    }
-
-    @Override
-    public boolean equals(final Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (getClass() != obj.getClass()) {
-            return false;
-        }
-        final SearchTerm other = (SearchTerm) obj;
-        if (this.hashCode != other.hashCode) {
-            return false;
-        }
-        return Arrays.equals(this.bytes, other.bytes);
-    }
-
-    @Override
-    public String toString() {
-        return new String(bytes);
-    }
-
-    public String toString(final Charset charset) {
-        return new String(bytes, charset);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/AhoCorasick.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/AhoCorasick.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/AhoCorasick.java
deleted file mode 100644
index 3b8afaf..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/AhoCorasick.java
+++ /dev/null
@@ -1,155 +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.search.ahocorasick;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.LinkedList;
-import java.util.Queue;
-import java.util.Set;
-
-import org.apache.nifi.util.search.Search;
-import org.apache.nifi.util.search.SearchTerm;
-
-public class AhoCorasick<T> implements Search<T> {
-
-    private Node root = null;
-
-    /**
-     * Constructs a new search object.
-     *
-     * @throws IllegalArgumentException if given terms are null or empty
-     */
-    public AhoCorasick() {
-    }
-
-    @Override
-    public void initializeDictionary(final Set<SearchTerm<T>> terms) {
-        if (root != null) {
-            throw new IllegalStateException();
-        }
-        root = new Node();
-        if (terms == null || terms.isEmpty()) {
-            throw new IllegalArgumentException();
-        }
-        for (final SearchTerm<T> term : terms) {
-            int i = 0;
-            Node nextNode = root;
-            while (true) {
-                nextNode = addMatch(term, i, nextNode);
-                if (nextNode == null) {
-                    break; //we're done
-                }
-                i++;
-            }
-        }
-        initialize();
-    }
-
-    private Node addMatch(final SearchTerm<T> term, final int offset, final Node current) {
-        final int index = term.get(offset);
-        boolean atEnd = (offset == (term.size() - 1));
-        if (current.getNeighbor(index) == null) {
-            if (atEnd) {
-                current.setNeighbor(new Node(term), index);
-                return null;
-            }
-            current.setNeighbor(new Node(), index);
-        } else if (atEnd) {
-            current.getNeighbor(index).setMatchingTerm(term);
-            return null;
-        }
-        return current.getNeighbor(index);
-    }
-
-    private void initialize() {
-        //perform bgs to build failure links
-        final Queue<Node> queue = new LinkedList<>();
-        queue.add(root);
-        root.setFailureNode(null);
-        while (!queue.isEmpty()) {
-            final Node current = queue.poll();
-            for (int i = 0; i < 256; i++) {
-                final Node next = current.getNeighbor(i);
-                if (next != null) {
-                    //traverse failure to get state
-                    Node fail = current.getFailureNode();
-                    while ((fail != null) && fail.getNeighbor(i) == null) {
-                        fail = fail.getFailureNode();
-                    }
-                    if (fail != null) {
-                        next.setFailureNode(fail.getNeighbor(i));
-                    } else {
-                        next.setFailureNode(root);
-                    }
-                    queue.add(next);
-                }
-            }
-        }
-    }
-
-    @Override
-    public SearchState search(final InputStream stream, final boolean findAll) throws IOException {
-        return search(stream, findAll, null);
-    }
-
-    private SearchState search(final InputStream stream, final boolean findAll, final SearchState state) throws IOException {
-        if (root == null) {
-            throw new IllegalStateException();
-        }
-        final SearchState<T> currentState = (state == null) ? new SearchState(root) : state;
-        if (!findAll && currentState.foundMatch()) {
-            throw new IllegalStateException("A match has already been found yet we're being asked to keep searching");
-        }
-        Node current = currentState.getCurrentNode();
-        int currentChar;
-        while ((currentChar = stream.read()) >= 0) {
-            currentState.incrementBytesRead(1L);
-            Node next = current.getNeighbor(currentChar);
-            if (next == null) {
-                next = current.getFailureNode();
-                while ((next != null) && next.getNeighbor(currentChar) == null) {
-                    next = next.getFailureNode();
-                }
-                if (next != null) {
-                    next = next.getNeighbor(currentChar);
-                } else {
-                    next = root;
-                }
-            }
-            if (next == null) {
-                throw new IllegalStateException("tree out of sync");
-            }
-            //Accept condition
-            if (next.hasMatch()) {
-                currentState.addResult(next.getMatchingTerm());
-            }
-            for (Node failNode = next.getFailureNode(); failNode != null; failNode = failNode.getFailureNode()) {
-                if (failNode.hasMatch()) {
-                    currentState.addResult(failNode.getMatchingTerm());
-                }
-            }
-            current = next;
-            if (currentState.foundMatch() && !findAll) {
-                break;//give up as soon as we have at least one match
-            }
-        }
-        currentState.setCurrentNode(current);
-        return currentState;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/Node.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/Node.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/Node.java
deleted file mode 100644
index 0ac325c..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/Node.java
+++ /dev/null
@@ -1,72 +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.search.ahocorasick;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.nifi.util.search.SearchTerm;
-
-/**
- *
- * @author
- */
-public class Node {
-
-    private final Map<Integer, Node> neighborMap;
-    private Node failureNode;
-    private SearchTerm<?> term;
-
-    Node(final SearchTerm<?> term) {
-        this();
-        this.term = term;
-    }
-
-    Node() {
-        neighborMap = new HashMap<>();
-        term = null;
-    }
-
-    void setFailureNode(final Node fail) {
-        failureNode = fail;
-    }
-
-    public Node getFailureNode() {
-        return failureNode;
-    }
-
-    public boolean hasMatch() {
-        return term != null;
-    }
-
-    void setMatchingTerm(final SearchTerm<?> term) {
-        this.term = term;
-    }
-
-    public SearchTerm<?> getMatchingTerm() {
-        return term;
-    }
-
-    public Node getNeighbor(final int index) {
-        return neighborMap.get(index);
-    }
-
-    void setNeighbor(final Node neighbor, final int index) {
-        neighborMap.put(index, neighbor);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/SearchState.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/SearchState.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/SearchState.java
deleted file mode 100644
index 6d36ad0..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/SearchState.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.search.ahocorasick;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.nifi.util.search.SearchTerm;
-
-public class SearchState<T> {
-
-    private Node currentNode;
-    private final Map<SearchTerm<T>, List<Long>> resultMap;
-    private long bytesRead;
-
-    SearchState(final Node rootNode) {
-        resultMap = new HashMap<>(5);
-        currentNode = rootNode;
-        bytesRead = 0L;
-    }
-
-    void incrementBytesRead(final long increment) {
-        bytesRead += increment;
-    }
-
-    void setCurrentNode(final Node curr) {
-        currentNode = curr;
-    }
-
-    public Node getCurrentNode() {
-        return currentNode;
-    }
-
-    public Map<SearchTerm<T>, List<Long>> getResults() {
-        return new HashMap<>(resultMap);
-    }
-
-    void addResult(final SearchTerm matchingTerm) {
-        final List<Long> indexes = (resultMap.containsKey(matchingTerm)) ? resultMap.get(matchingTerm) : new ArrayList<Long>(5);
-        indexes.add(bytesRead);
-        resultMap.put(matchingTerm, indexes);
-    }
-
-    public boolean foundMatch() {
-        return !resultMap.isEmpty();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/EntityAccess.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/EntityAccess.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/EntityAccess.java
deleted file mode 100644
index 2b95897..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/EntityAccess.java
+++ /dev/null
@@ -1,26 +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.timebuffer;
-
-public interface EntityAccess<T> {
-
-    T aggregate(T oldValue, T toAdd);
-
-    T createNew();
-
-    long getTimestamp(T entity);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/LongEntityAccess.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/LongEntityAccess.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/LongEntityAccess.java
deleted file mode 100644
index 193abc6..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/LongEntityAccess.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.util.timebuffer;
-
-public class LongEntityAccess implements EntityAccess<TimestampedLong> {
-
-    @Override
-    public TimestampedLong aggregate(TimestampedLong oldValue, TimestampedLong toAdd) {
-        if (oldValue == null && toAdd == null) {
-            return new TimestampedLong(0L);
-        } else if (oldValue == null) {
-            return toAdd;
-        } else if (toAdd == null) {
-            return oldValue;
-        }
-
-        return new TimestampedLong(oldValue.getValue() + toAdd.getValue());
-    }
-
-    @Override
-    public TimestampedLong createNew() {
-        return new TimestampedLong(0L);
-    }
-
-    @Override
-    public long getTimestamp(TimestampedLong entity) {
-        return entity == null ? 0L : entity.getTimestamp();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/TimedBuffer.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/TimedBuffer.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/TimedBuffer.java
deleted file mode 100644
index dd8e523..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/TimedBuffer.java
+++ /dev/null
@@ -1,114 +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.timebuffer;
-
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
-public class TimedBuffer<T> {
-
-    private final int numBins;
-    private final EntitySum<T>[] bins;
-    private final EntityAccess<T> entityAccess;
-    private final TimeUnit binPrecision;
-
-    @SuppressWarnings("unchecked")
-    public TimedBuffer(final TimeUnit binPrecision, final int numBins, final EntityAccess<T> accessor) {
-        this.binPrecision = binPrecision;
-        this.numBins = numBins + 1;
-        this.bins = new EntitySum[this.numBins];
-        for (int i = 0; i < this.numBins; i++) {
-            this.bins[i] = new EntitySum<>(binPrecision, numBins, accessor);
-        }
-        this.entityAccess = accessor;
-    }
-
-    public T add(final T entity) {
-        final int binIdx = (int) (binPrecision.convert(System.currentTimeMillis(), TimeUnit.MILLISECONDS) % numBins);
-        final EntitySum<T> sum = bins[binIdx];
-
-        return sum.addOrReset(entity);
-    }
-
-    public T getAggregateValue(final long sinceEpochMillis) {
-        final int startBinIdx = (int) (binPrecision.convert(sinceEpochMillis, TimeUnit.MILLISECONDS) % numBins);
-
-        T total = null;
-        for (int i = 0; i < numBins; i++) {
-            int binIdx = (startBinIdx + i) % numBins;
-            final EntitySum<T> bin = bins[binIdx];
-
-            if (!bin.isExpired()) {
-                total = entityAccess.aggregate(total, bin.getValue());
-            }
-        }
-
-        return total;
-    }
-
-    private static class EntitySum<S> {
-
-        private final EntityAccess<S> entityAccess;
-        private final AtomicReference<S> ref = new AtomicReference<>();
-        private final TimeUnit binPrecision;
-        private final int numConfiguredBins;
-
-        public EntitySum(final TimeUnit binPrecision, final int numConfiguredBins, final EntityAccess<S> aggregator) {
-            this.binPrecision = binPrecision;
-            this.entityAccess = aggregator;
-            this.numConfiguredBins = numConfiguredBins;
-        }
-
-        private S add(final S event) {
-            S newValue;
-            S value;
-            do {
-                value = ref.get();
-                newValue = entityAccess.aggregate(value, event);
-            } while (!ref.compareAndSet(value, newValue));
-
-            return newValue;
-        }
-
-        public S getValue() {
-            return ref.get();
-        }
-
-        public boolean isExpired() {
-            // entityAccess.getTimestamp(curValue) represents the time at which the current value
-            // was last updated. If the last value is less than current time - 1 binPrecision, then it
-            // means that we've rolled over and need to reset the value.
-            final long maxExpectedTimePeriod = System.currentTimeMillis() - TimeUnit.MILLISECONDS.convert(numConfiguredBins, binPrecision);
-
-            final S curValue = ref.get();
-            return (entityAccess.getTimestamp(curValue) < maxExpectedTimePeriod);
-        }
-
-        public S addOrReset(final S event) {
-            // entityAccess.getTimestamp(curValue) represents the time at which the current value
-            // was last updated. If the last value is less than current time - 1 binPrecision, then it
-            // means that we've rolled over and need to reset the value.
-            final long maxExpectedTimePeriod = System.currentTimeMillis() - TimeUnit.MILLISECONDS.convert(1, binPrecision);
-
-            final S curValue = ref.get();
-            if (entityAccess.getTimestamp(curValue) < maxExpectedTimePeriod) {
-                ref.compareAndSet(curValue, entityAccess.createNew());
-            }
-            return add(event);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/TimestampedLong.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/TimestampedLong.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/TimestampedLong.java
deleted file mode 100644
index 07d31ea..0000000
--- a/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/TimestampedLong.java
+++ /dev/null
@@ -1,35 +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.timebuffer;
-
-public class TimestampedLong {
-
-    private final Long value;
-    private final long timestamp = System.currentTimeMillis();
-
-    public TimestampedLong(final Long value) {
-        this.value = value;
-    }
-
-    public Long getValue() {
-        return value;
-    }
-
-    public long getTimestamp() {
-        return timestamp;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/test/java/org/apache/nifi/remote/io/TestCompressionInputOutputStreams.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/test/java/org/apache/nifi/remote/io/TestCompressionInputOutputStreams.java b/commons/nifi-utils/src/test/java/org/apache/nifi/remote/io/TestCompressionInputOutputStreams.java
deleted file mode 100644
index bd30a96..0000000
--- a/commons/nifi-utils/src/test/java/org/apache/nifi/remote/io/TestCompressionInputOutputStreams.java
+++ /dev/null
@@ -1,153 +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.remote.io;
-
-import static org.junit.Assert.assertTrue;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Arrays;
-
-import org.apache.nifi.remote.io.CompressionInputStream;
-import org.apache.nifi.remote.io.CompressionOutputStream;
-
-import org.junit.Test;
-
-public class TestCompressionInputOutputStreams {
-
-    @Test
-    public void testSimple() throws IOException {
-        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-
-        final byte[] data = "Hello, World!".getBytes("UTF-8");
-
-        final CompressionOutputStream cos = new CompressionOutputStream(baos);
-        cos.write(data);
-        cos.flush();
-        cos.close();
-
-        final byte[] compressedBytes = baos.toByteArray();
-        final CompressionInputStream cis = new CompressionInputStream(new ByteArrayInputStream(compressedBytes));
-        final byte[] decompressed = readFully(cis);
-
-        assertTrue(Arrays.equals(data, decompressed));
-    }
-
-    @Test
-    public void testDataLargerThanBuffer() throws IOException {
-        final String str = "The quick brown fox jumps over the lazy dog\r\n\n\n\r";
-
-        final StringBuilder sb = new StringBuilder();
-        for (int i = 0; i < 100; i++) {
-            sb.append(str);
-        }
-        final byte[] data = sb.toString().getBytes("UTF-8");
-
-        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-
-        final CompressionOutputStream cos = new CompressionOutputStream(baos, 8192);
-        cos.write(data);
-        cos.flush();
-        cos.close();
-
-        final byte[] compressedBytes = baos.toByteArray();
-        final CompressionInputStream cis = new CompressionInputStream(new ByteArrayInputStream(compressedBytes));
-        final byte[] decompressed = readFully(cis);
-
-        assertTrue(Arrays.equals(data, decompressed));
-    }
-
-    @Test
-    public void testDataLargerThanBufferWhileFlushing() throws IOException {
-        final String str = "The quick brown fox jumps over the lazy dog\r\n\n\n\r";
-        final byte[] data = str.getBytes("UTF-8");
-
-        final StringBuilder sb = new StringBuilder();
-        final byte[] data1024;
-
-        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-
-        final CompressionOutputStream cos = new CompressionOutputStream(baos, 8192);
-        for (int i = 0; i < 1024; i++) {
-            cos.write(data);
-            cos.flush();
-            sb.append(str);
-        }
-        cos.close();
-        data1024 = sb.toString().getBytes("UTF-8");
-
-        final byte[] compressedBytes = baos.toByteArray();
-        final CompressionInputStream cis = new CompressionInputStream(new ByteArrayInputStream(compressedBytes));
-        final byte[] decompressed = readFully(cis);
-
-        assertTrue(Arrays.equals(data1024, decompressed));
-    }
-
-    @Test
-    public void testSendingMultipleFilesBackToBackOnSameStream() throws IOException {
-        final String str = "The quick brown fox jumps over the lazy dog\r\n\n\n\r";
-        final byte[] data = str.getBytes("UTF-8");
-
-        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-
-        final CompressionOutputStream cos = new CompressionOutputStream(baos, 8192);
-        for (int i = 0; i < 512; i++) {
-            cos.write(data);
-            cos.flush();
-        }
-        cos.close();
-
-        final CompressionOutputStream cos2 = new CompressionOutputStream(baos, 8192);
-        for (int i = 0; i < 512; i++) {
-            cos2.write(data);
-            cos2.flush();
-        }
-        cos2.close();
-
-        final byte[] data512;
-        final StringBuilder sb = new StringBuilder();
-        for (int i = 0; i < 512; i++) {
-            sb.append(str);
-        }
-        data512 = sb.toString().getBytes("UTF-8");
-
-        final byte[] compressedBytes = baos.toByteArray();
-        final ByteArrayInputStream bais = new ByteArrayInputStream(compressedBytes);
-
-        final CompressionInputStream cis = new CompressionInputStream(bais);
-        final byte[] decompressed = readFully(cis);
-        assertTrue(Arrays.equals(data512, decompressed));
-
-        final CompressionInputStream cis2 = new CompressionInputStream(bais);
-        final byte[] decompressed2 = readFully(cis2);
-        assertTrue(Arrays.equals(data512, decompressed2));
-    }
-
-    private byte[] readFully(final InputStream in) throws IOException {
-        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-
-        final byte[] buffer = new byte[65536];
-        int len;
-        while ((len = in.read(buffer)) >= 0) {
-            baos.write(buffer, 0, len);
-        }
-
-        return baos.toByteArray();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/test/java/org/apache/nifi/stream/io/TestLeakyBucketThrottler.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/test/java/org/apache/nifi/stream/io/TestLeakyBucketThrottler.java b/commons/nifi-utils/src/test/java/org/apache/nifi/stream/io/TestLeakyBucketThrottler.java
deleted file mode 100644
index 52bd8de..0000000
--- a/commons/nifi-utils/src/test/java/org/apache/nifi/stream/io/TestLeakyBucketThrottler.java
+++ /dev/null
@@ -1,147 +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.stream.io;
-
-import org.apache.nifi.stream.io.ByteArrayInputStream;
-import org.apache.nifi.stream.io.ByteArrayOutputStream;
-import org.apache.nifi.stream.io.LeakyBucketStreamThrottler;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.junit.Ignore;
-import org.junit.Test;
-
-@Ignore("Tests are time-based")
-public class TestLeakyBucketThrottler {
-
-    @Test(timeout = 10000)
-    public void testOutputStreamInterface() throws IOException {
-        // throttle rate at 1 MB/sec
-        final LeakyBucketStreamThrottler throttler = new LeakyBucketStreamThrottler(1024 * 1024);
-
-        final byte[] data = new byte[1024 * 1024 * 4];
-        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        final OutputStream throttledOut = throttler.newThrottledOutputStream(baos);
-
-        final long start = System.currentTimeMillis();
-        throttledOut.write(data);
-        throttler.close();
-        final long millis = System.currentTimeMillis() - start;
-        // should take 4 sec give or take
-        assertTrue(millis > 3000);
-        assertTrue(millis < 6000);
-    }
-
-    @Test(timeout = 10000)
-    public void testInputStreamInterface() throws IOException {
-        // throttle rate at 1 MB/sec
-        final LeakyBucketStreamThrottler throttler = new LeakyBucketStreamThrottler(1024 * 1024);
-
-        final byte[] data = new byte[1024 * 1024 * 4];
-        final ByteArrayInputStream bais = new ByteArrayInputStream(data);
-        final InputStream throttledIn = throttler.newThrottledInputStream(bais);
-
-        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-
-        final byte[] buffer = new byte[4096];
-        final long start = System.currentTimeMillis();
-        int len;
-        while ((len = throttledIn.read(buffer)) > 0) {
-            baos.write(buffer, 0, len);
-        }
-        throttler.close();
-        final long millis = System.currentTimeMillis() - start;
-        // should take 4 sec give or take
-        assertTrue(millis > 3000);
-        assertTrue(millis < 6000);
-        baos.close();
-    }
-
-    @Test(timeout = 10000)
-    public void testDirectInterface() throws IOException, InterruptedException {
-        // throttle rate at 1 MB/sec
-        final LeakyBucketStreamThrottler throttler = new LeakyBucketStreamThrottler(1024 * 1024);
-
-        // create 3 threads, each sending ~2 MB
-        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        final List<Thread> threads = new ArrayList<Thread>();
-        for (int i = 0; i < 3; i++) {
-            final Thread t = new WriterThread(i, throttler, baos);
-            threads.add(t);
-        }
-
-        final long start = System.currentTimeMillis();
-        for (final Thread t : threads) {
-            t.start();
-        }
-
-        for (final Thread t : threads) {
-            t.join();
-        }
-        final long elapsed = System.currentTimeMillis() - start;
-
-        throttler.close();
-
-        // To send 15 MB, it should have taken at least 5 seconds and no more than 7 seconds, to
-        // allow for busy-ness and the fact that we could write a tiny bit more than the limit.
-        assertTrue(elapsed > 5000);
-        assertTrue(elapsed < 7000);
-
-        // ensure bytes were copied out appropriately
-        assertEquals(3 * (2 * 1024 * 1024 + 1), baos.getBufferLength());
-        assertEquals((byte) 'A', baos.getUnderlyingBuffer()[baos.getBufferLength() - 1]);
-    }
-
-    private static class WriterThread extends Thread {
-
-        private final int idx;
-        private final byte[] data = new byte[1024 * 1024 * 2 + 1];
-        private final LeakyBucketStreamThrottler throttler;
-        private final OutputStream out;
-
-        public WriterThread(final int idx, final LeakyBucketStreamThrottler throttler, final OutputStream out) {
-            this.idx = idx;
-            this.throttler = throttler;
-            this.out = out;
-            this.data[this.data.length - 1] = (byte) 'A';
-        }
-
-        @Override
-        public void run() {
-            long startMillis = System.currentTimeMillis();
-            long bytesWritten = 0L;
-            try {
-                throttler.copy(new ByteArrayInputStream(data), out);
-            } catch (IOException e) {
-                e.printStackTrace();
-                return;
-            }
-            long now = System.currentTimeMillis();
-            long millisElapsed = now - startMillis;
-            bytesWritten += data.length;
-            float bytesPerSec = (float) bytesWritten / (float) millisElapsed * 1000F;
-            System.out.println(idx + " : copied data at a rate of " + bytesPerSec + " bytes/sec");
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/test/java/org/apache/nifi/util/TestNaiveSearchRingBuffer.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/test/java/org/apache/nifi/util/TestNaiveSearchRingBuffer.java b/commons/nifi-utils/src/test/java/org/apache/nifi/util/TestNaiveSearchRingBuffer.java
deleted file mode 100644
index 0838e96..0000000
--- a/commons/nifi-utils/src/test/java/org/apache/nifi/util/TestNaiveSearchRingBuffer.java
+++ /dev/null
@@ -1,72 +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 static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import org.junit.Test;
-
-public class TestNaiveSearchRingBuffer {
-
-    @Test
-    public void testAddAndCompare() {
-        final byte[] pattern = new byte[]{
-            '\r', '0', 38, 48
-        };
-
-        final byte[] search = new byte[]{
-            '\r', '0', 38, 58, 58, 83, 78, '\r', '0', 38, 48, 83, 92, 78, 4, 38
-        };
-
-        final NaiveSearchRingBuffer circ = new NaiveSearchRingBuffer(pattern);
-        int counter = -1;
-        for (final byte b : search) {
-            counter++;
-            final boolean matched = circ.addAndCompare(b);
-            if (counter == 10) {
-                assertTrue(matched);
-            } else {
-                assertFalse(matched);
-            }
-        }
-    }
-
-    @Test
-    public void testGetOldestByte() {
-        final byte[] pattern = new byte[]{
-            '\r', '0', 38, 48
-        };
-
-        final byte[] search = new byte[]{
-            '\r', '0', 38, 58, 58, 83, 78, (byte) 223, (byte) 227, (byte) 250, '\r', '0', 38, 48, 83, 92, 78, 4, 38
-        };
-
-        final NaiveSearchRingBuffer circ = new NaiveSearchRingBuffer(pattern);
-        int counter = -1;
-        for (final byte b : search) {
-            counter++;
-            final boolean matched = circ.addAndCompare(b);
-            if (counter == 13) {
-                assertTrue(matched);
-            } else {
-                assertFalse(matched);
-            }
-        }
-    }
-
-}


[22/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusHistoryDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusHistoryDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusHistoryDTO.java
deleted file mode 100644
index c8b05f3..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusHistoryDTO.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.api.dto.status;
-
-import javax.xml.bind.annotation.XmlType;
-import org.apache.nifi.web.api.dto.NodeDTO;
-
-/**
- * DTO for serializing the status history for a particular node.
- */
-@XmlType(name = "nodeStatusHistory")
-public class NodeStatusHistoryDTO {
-
-    private NodeDTO node;
-    private StatusHistoryDTO statusHistory;
-
-    /**
-     * The node.
-     *
-     * @return
-     */
-    public NodeDTO getNode() {
-        return node;
-    }
-
-    public void setNode(NodeDTO node) {
-        this.node = node;
-    }
-
-    /**
-     * The processor status history.
-     *
-     * @return
-     */
-    public StatusHistoryDTO getStatusHistory() {
-        return statusHistory;
-    }
-
-    public void setStatusHistory(StatusHistoryDTO statusHistory) {
-        this.statusHistory = statusHistory;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusDTO.java
deleted file mode 100644
index c32efdb..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusDTO.java
+++ /dev/null
@@ -1,142 +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.api.dto.status;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * The status for a port in this NiFi.
- */
-@XmlType(name = "portStatus")
-public class PortStatusDTO extends StatusDTO {
-
-    private String id;
-    private String groupId;
-    private String name;
-    private Integer activeThreadCount;
-    private String input;
-    private String output;
-    private Boolean transmitting;
-    private String runStatus;
-
-    /**
-     * Whether this port has incoming or outgoing connections to a remote NiFi.
-     *
-     * @return
-     */
-    public Boolean isTransmitting() {
-        return transmitting;
-    }
-
-    public void setTransmitting(Boolean transmitting) {
-        this.transmitting = transmitting;
-    }
-
-    /**
-     * The active thread count for this port.
-     *
-     * @return
-     */
-    public Integer getActiveThreadCount() {
-        return activeThreadCount;
-    }
-
-    public void setActiveThreadCount(Integer activeThreadCount) {
-        this.activeThreadCount = activeThreadCount;
-    }
-
-    /**
-     * The id of this port.
-     *
-     * @return
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The id of the group this port resides in.
-     *
-     * @return
-     */
-    public String getGroupId() {
-        return groupId;
-    }
-
-    public void setGroupId(String groupId) {
-        this.groupId = groupId;
-    }
-
-    /**
-     * The name of this port.
-     *
-     * @return
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * The run status of this port.
-     *
-     * @return
-     */
-    public String getRunStatus() {
-        return runStatus;
-    }
-
-    public void setRunStatus(String runStatus) {
-        this.runStatus = runStatus;
-    }
-
-    /**
-     * The total count and size of flow files that have been accepted in the
-     * last five minutes.
-     *
-     * @return The total processed
-     */
-    public String getInput() {
-        return input;
-    }
-
-    public void setInput(String input) {
-        this.input = input;
-    }
-
-    /**
-     * The total count and size of flow files that have been processed in the
-     * last five minutes.
-     *
-     * @return The total output
-     */
-    public String getOutput() {
-        return output;
-    }
-
-    public void setOutput(String output) {
-        this.output = output;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.java
deleted file mode 100644
index 4fa2b64..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.java
+++ /dev/null
@@ -1,244 +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.api.dto.status;
-
-import java.util.Collection;
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * The status for a process group in this NiFi.
- */
-@XmlType(name = "processGroupStatus")
-public class ProcessGroupStatusDTO extends StatusDTO {
-
-    private String id;
-    private String name;
-    private Collection<ConnectionStatusDTO> connectionStatus;
-    private Collection<ProcessorStatusDTO> processorStatus;
-    private Collection<ProcessGroupStatusDTO> processGroupStatus;
-    private Collection<RemoteProcessGroupStatusDTO> remoteProcessGroupStatus;
-    private Collection<PortStatusDTO> inputPortStatus;
-    private Collection<PortStatusDTO> outputPortStatus;
-
-    private String input;
-    private String queued;
-    private String read;
-    private String written;
-    private String output;
-    private Integer activeThreadCount;
-    private Date statsLastRefreshed;
-
-    /**
-     * The id for the process group.
-     *
-     * @return The id for the process group
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The name of this process group.
-     *
-     * @return
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * The active thread count for this process group.
-     *
-     * @return
-     */
-    public Integer getActiveThreadCount() {
-        return activeThreadCount;
-    }
-
-    public void setActiveThreadCount(Integer activeThreadCount) {
-        this.activeThreadCount = activeThreadCount;
-    }
-
-    /**
-     * The status of all connections in this process group.
-     *
-     * @return The status of all connections
-     */
-    public Collection<ConnectionStatusDTO> getConnectionStatus() {
-        return connectionStatus;
-    }
-
-    public void setConnectionStatus(Collection<ConnectionStatusDTO> connectionStatus) {
-        this.connectionStatus = connectionStatus;
-    }
-
-    /**
-     * The status of all process groups in this process group.
-     *
-     * @return The status of all process groups
-     */
-    public Collection<ProcessGroupStatusDTO> getProcessGroupStatus() {
-        return processGroupStatus;
-    }
-
-    public void setProcessGroupStatus(Collection<ProcessGroupStatusDTO> processGroupStatus) {
-        this.processGroupStatus = processGroupStatus;
-    }
-
-    /**
-     * The status of all remote process groups in this process group.
-     *
-     * @return The status of all remote process groups
-     */
-    public Collection<RemoteProcessGroupStatusDTO> getRemoteProcessGroupStatus() {
-        return remoteProcessGroupStatus;
-    }
-
-    public void setRemoteProcessGroupStatus(final Collection<RemoteProcessGroupStatusDTO> remoteProcessGroupStatus) {
-        this.remoteProcessGroupStatus = remoteProcessGroupStatus;
-    }
-
-    /**
-     * The status of all processors in this process group.
-     *
-     * @return The status of all processors
-     */
-    public Collection<ProcessorStatusDTO> getProcessorStatus() {
-        return processorStatus;
-    }
-
-    public void setProcessorStatus(Collection<ProcessorStatusDTO> processorStatus) {
-        this.processorStatus = processorStatus;
-    }
-
-    /**
-     * The status of all input ports in this process group.
-     *
-     * @return The status of all input ports
-     */
-    public Collection<PortStatusDTO> getInputPortStatus() {
-        return inputPortStatus;
-    }
-
-    public void setInputPortStatus(Collection<PortStatusDTO> inputPortStatus) {
-        this.inputPortStatus = inputPortStatus;
-    }
-
-    /**
-     * The status of all output ports in this process group.
-     *
-     * @return The status of all output ports
-     */
-    public Collection<PortStatusDTO> getOutputPortStatus() {
-        return outputPortStatus;
-    }
-
-    public void setOutputPortStatus(Collection<PortStatusDTO> outputPortStatus) {
-        this.outputPortStatus = outputPortStatus;
-    }
-
-    /**
-     * The output stats for this process group.
-     *
-     * @return The output stats
-     */
-    public String getOutput() {
-        return output;
-    }
-
-    public void setOutput(String output) {
-        this.output = output;
-    }
-
-    /**
-     * The queued stats for this process group.
-     *
-     * @return The queued stats
-     */
-    public String getQueued() {
-        return queued;
-    }
-
-    public void setQueued(String queued) {
-        this.queued = queued;
-    }
-
-    /**
-     * The read stats for this process group.
-     *
-     * @return The read stats
-     */
-    public String getRead() {
-        return read;
-    }
-
-    public void setRead(String read) {
-        this.read = read;
-    }
-
-    /**
-     * The written stats for this process group.
-     *
-     * @return The written stats
-     */
-    public String getWritten() {
-        return written;
-    }
-
-    public void setWritten(String written) {
-        this.written = written;
-    }
-
-    /**
-     * The input stats for this process group.
-     *
-     * @return The input stats
-     */
-    public String getInput() {
-        return input;
-    }
-
-    public void setInput(String input) {
-        this.input = input;
-    }
-
-    /**
-     * When the status for this process group was calculated.
-     *
-     * @return The the status was calculated
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    public Date getStatsLastRefreshed() {
-        return statsLastRefreshed;
-    }
-
-    public void setStatsLastRefreshed(Date statsLastRefreshed) {
-        this.statsLastRefreshed = statsLastRefreshed;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessorStatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessorStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessorStatusDTO.java
deleted file mode 100644
index 334adad..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessorStatusDTO.java
+++ /dev/null
@@ -1,203 +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.api.dto.status;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * DTO for serializing the status of a processor.
- */
-@XmlType(name = "processorStatus")
-public class ProcessorStatusDTO extends StatusDTO {
-
-    private String id;
-    private String groupId;
-    private String name;
-    private String type;
-    private String runStatus;
-
-    private String read;
-    private String written;
-
-    private String input;
-    private String output;
-
-    private String tasks;
-    private String tasksDuration;
-    private Integer activeThreadCount;
-
-    /* getters / setters */
-    /**
-     * The id for the processor.
-     *
-     * @return The processor id
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The name of the processor.
-     *
-     * @return The processor name
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * The type of the processor.
-     *
-     * @return The processor type
-     */
-    public String getType() {
-        return type;
-    }
-
-    public void setType(String type) {
-        this.type = type;
-    }
-
-    /**
-     * The run status of this processor.
-     *
-     * @return
-     */
-    public String getRunStatus() {
-        return runStatus;
-    }
-
-    public void setRunStatus(String runStatus) {
-        this.runStatus = runStatus;
-    }
-
-    /**
-     * The total count and size of flow files that have been accepted in the
-     * last five minutes.
-     *
-     * @return The total processed
-     */
-    public String getInput() {
-        return input;
-    }
-
-    public void setInput(String input) {
-        this.input = input;
-    }
-
-    /**
-     * The number of bytes read.
-     *
-     * @return
-     */
-    public String getRead() {
-        return read;
-    }
-
-    public void setRead(String read) {
-        this.read = read;
-    }
-
-    /**
-     * The number of bytes written.
-     *
-     * @return
-     */
-    public String getWritten() {
-        return written;
-    }
-
-    public void setWritten(String written) {
-        this.written = written;
-    }
-
-    /**
-     * The ID of the Process Group to which this processor belongs.
-     *
-     * @return the ID of the Process Group to which this processor belongs.
-     */
-    public String getGroupId() {
-        return groupId;
-    }
-
-    public void setGroupId(final String groupId) {
-        this.groupId = groupId;
-    }
-
-    /**
-     * The total count and size of flow files that have been processed in the
-     * last five minutes.
-     *
-     * @return The total output
-     */
-    public String getOutput() {
-        return output;
-    }
-
-    public void setOutput(String output) {
-        this.output = output;
-    }
-
-    /**
-     * The number of threads currently running for this Processor
-     *
-     * @return
-     */
-    public Integer getActiveThreadCount() {
-        return activeThreadCount;
-    }
-
-    public void setActiveThreadCount(Integer threadCount) {
-        this.activeThreadCount = threadCount;
-    }
-
-    /**
-     * The number of task this connectable has had over the last 5 minutes.
-     *
-     * @return
-     */
-    public String getTasks() {
-        return tasks;
-    }
-
-    public void setTasks(String tasks) {
-        this.tasks = tasks;
-    }
-
-    /**
-     * The total duration of all tasks for this connectable over the last 5
-     * minutes.
-     *
-     * @return
-     */
-    public String getTasksDuration() {
-        return tasksDuration;
-    }
-
-    public void setTasksDuration(String tasksDuration) {
-        this.tasksDuration = tasksDuration;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemotePortStatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemotePortStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemotePortStatusDTO.java
deleted file mode 100644
index a8fcc9f..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemotePortStatusDTO.java
+++ /dev/null
@@ -1,98 +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.api.dto.status;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * The status of a Port on a remote NiFi instance.
- */
-@XmlType(name = "remotePortStatus")
-public class RemotePortStatusDTO {
-
-    private String id;
-    private String connectionId;
-    private String name;
-    private Boolean running;
-    private Boolean exists;
-
-    /**
-     * The id of the connection this remote port is connected to.
-     *
-     * @return
-     */
-    public String getConnectionId() {
-        return connectionId;
-    }
-
-    public void setConnectionId(String connectionId) {
-        this.connectionId = connectionId;
-    }
-
-    /**
-     * The id of the remote port.
-     *
-     * @return
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The name of the remote port.
-     *
-     * @return
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * Whether or not the remote port exists.
-     *
-     * @return
-     */
-    public Boolean getExists() {
-        return exists;
-    }
-
-    public void setExists(Boolean exists) {
-        this.exists = exists;
-    }
-
-    /**
-     * Whether or not the remote port is running.
-     *
-     * @return
-     */
-    public Boolean getRunning() {
-        return running;
-    }
-
-    public void setRunning(Boolean running) {
-        this.running = running;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.java
deleted file mode 100644
index 363d4da..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.java
+++ /dev/null
@@ -1,159 +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.api.dto.status;
-
-import java.util.List;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * The status of a remote process group in this NiFi.
- */
-@XmlType(name = "remoteProcessGroupStatus")
-public class RemoteProcessGroupStatusDTO extends StatusDTO {
-
-    private String id;
-    private String groupId;
-    private String name;
-    private String targetUri;
-    private String transmissionStatus;
-    private Integer activeThreadCount;
-
-    private List<String> authorizationIssues;
-
-    private String sent;
-    private String received;
-
-    /**
-     * The id for the remote process group.
-     *
-     * @return The id for the remote process group
-     */
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * The id of the group this remote process group is in.
-     *
-     * @return
-     */
-    public String getGroupId() {
-        return groupId;
-    }
-
-    public void setGroupId(String groupId) {
-        this.groupId = groupId;
-    }
-
-    /**
-     * The URI of the target system.
-     *
-     * @return
-     */
-    public String getTargetUri() {
-        return targetUri;
-    }
-
-    public void setTargetUri(String targetUri) {
-        this.targetUri = targetUri;
-    }
-
-    /**
-     * The name of this remote process group.
-     *
-     * @return
-     */
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * The transmission status of this remote process group.
-     *
-     * @return
-     */
-    public String getTransmissionStatus() {
-        return transmissionStatus;
-    }
-
-    public void setTransmissionStatus(String transmissionStatus) {
-        this.transmissionStatus = transmissionStatus;
-    }
-
-    /**
-     * The number of active threads.
-     *
-     * @return
-     */
-    public Integer getActiveThreadCount() {
-        return activeThreadCount;
-    }
-
-    public void setActiveThreadCount(Integer activeThreadCount) {
-        this.activeThreadCount = activeThreadCount;
-    }
-
-    /**
-     * Returns any remote authorization issues for this remote process group.
-     *
-     * @return
-     */
-    public List<String> getAuthorizationIssues() {
-        return authorizationIssues;
-    }
-
-    public void setAuthorizationIssues(List<String> authorizationIssues) {
-        this.authorizationIssues = authorizationIssues;
-    }
-
-    /**
-     * Formatted description of the amount of data sent to this remote process
-     * group.
-     *
-     * @return
-     */
-    public String getSent() {
-        return sent;
-    }
-
-    public void setSent(String sent) {
-        this.sent = sent;
-    }
-
-    /**
-     * Formatted description of the amount of data received from this remote
-     * process group.
-     *
-     * @return
-     */
-    public String getReceived() {
-        return received;
-    }
-
-    public void setReceived(String received) {
-        this.received = received;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDTO.java
deleted file mode 100644
index e4e7a45..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDTO.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.web.api.dto.status;
-
-import java.util.List;
-import javax.xml.bind.annotation.XmlType;
-import org.apache.nifi.web.api.dto.BulletinDTO;
-
-/**
- * The status of a component in this NiFi.
- */
-@XmlType(name = "status")
-public abstract class StatusDTO {
-
-    private List<BulletinDTO> bulletins;
-
-    /**
-     * Bulletins for this component.
-     *
-     * @return
-     */
-    public List<BulletinDTO> getBulletins() {
-        return bulletins;
-    }
-
-    public void setBulletins(List<BulletinDTO> bulletins) {
-        this.bulletins = bulletins;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDescriptorDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDescriptorDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDescriptorDTO.java
deleted file mode 100644
index a203e10..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDescriptorDTO.java
+++ /dev/null
@@ -1,101 +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.api.dto.status;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * DTO for serializing a status descriptor.
- */
-@XmlType(name = "statusDescriptor")
-public class StatusDescriptorDTO {
-
-    public enum Formatter {
-
-        COUNT,
-        DURATION,
-        DATA_SIZE
-    };
-
-    private String field;
-    private String label;
-    private String description;
-    private String formatter;
-
-    public StatusDescriptorDTO() {
-    }
-
-    public StatusDescriptorDTO(final String field, final String label, final String description, final String formatter) {
-        this.field = field;
-        this.label = label;
-        this.description = description;
-        this.formatter = formatter;
-    }
-
-    /**
-     * The name of this status field.
-     *
-     * @return
-     */
-    public String getField() {
-        return field;
-    }
-
-    public void setField(String field) {
-        this.field = field;
-    }
-
-    /**
-     * The label of this status field.
-     *
-     * @return
-     */
-    public String getLabel() {
-        return label;
-    }
-
-    public void setLabel(String label) {
-        this.label = label;
-    }
-
-    /**
-     * The description of this status field.
-     *
-     * @return
-     */
-    public String getDescription() {
-        return description;
-    }
-
-    public void setDescription(String description) {
-        this.description = description;
-    }
-
-    /**
-     * The formatter for this descriptor.
-     *
-     * @return
-     */
-    public String getFormatter() {
-        return formatter;
-    }
-
-    public void setFormatter(String formatter) {
-        this.formatter = formatter;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDTO.java
deleted file mode 100644
index 5b97c16..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDTO.java
+++ /dev/null
@@ -1,92 +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.api.dto.status;
-
-import java.util.Date;
-import java.util.LinkedHashMap;
-import java.util.List;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * History status for a component in this NiFi.
- */
-@XmlType(name = "statusHistory")
-public class StatusHistoryDTO {
-
-    private Date generated;
-
-    private LinkedHashMap<String, String> details;
-
-    private List<StatusDescriptorDTO> fieldDescriptors;
-    private List<StatusSnapshotDTO> statusSnapshots;
-
-    /**
-     * When this status history was generated.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    public Date getGenerated() {
-        return generated;
-    }
-
-    public void setGenerated(Date generated) {
-        this.generated = generated;
-    }
-
-    /**
-     * The component details for this status history.
-     *
-     * @return
-     */
-    public LinkedHashMap<String, String> getDetails() {
-        return details;
-    }
-
-    public void setDetails(LinkedHashMap<String, String> details) {
-        this.details = details;
-    }
-
-    /**
-     * Descriptors for each supported status field.
-     *
-     * @return
-     */
-    public List<StatusDescriptorDTO> getFieldDescriptors() {
-        return fieldDescriptors;
-    }
-
-    public void setFieldDescriptors(List<StatusDescriptorDTO> fieldDescriptors) {
-        this.fieldDescriptors = fieldDescriptors;
-    }
-
-    /**
-     * The status snapshots.
-     *
-     * @return
-     */
-    public List<StatusSnapshotDTO> getStatusSnapshots() {
-        return statusSnapshots;
-    }
-
-    public void setStatusSnapshots(List<StatusSnapshotDTO> statusSnapshots) {
-        this.statusSnapshots = statusSnapshots;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDetailDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDetailDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDetailDTO.java
deleted file mode 100644
index b9d16bb..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDetailDTO.java
+++ /dev/null
@@ -1,56 +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.api.dto.status;
-
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * Detail of a status history metric.
- */
-@XmlType(name = "statusHistoryDetail")
-public class StatusHistoryDetailDTO {
-
-    private String label;
-    private String value;
-
-    /**
-     * The label for this status detail.
-     *
-     * @return
-     */
-    public String getLabel() {
-        return label;
-    }
-
-    public void setLabel(String label) {
-        this.label = label;
-    }
-
-    /**
-     * The value for this status detail.
-     *
-     * @return
-     */
-    public String getValue() {
-        return value;
-    }
-
-    public void setValue(String value) {
-        this.value = value;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusSnapshotDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusSnapshotDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusSnapshotDTO.java
deleted file mode 100644
index cc6894a..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusSnapshotDTO.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.web.api.dto.status;
-
-import java.util.Date;
-import java.util.Map;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * A snapshot of the status at a given time.
- */
-@XmlType(name = "statusSnapshot")
-public class StatusSnapshotDTO {
-
-    private Date timestamp;
-    private Map<String, Long> statusMetrics;
-
-    /**
-     * The timestamp of this snapshot.
-     *
-     * @return
-     */
-    public Date getTimestamp() {
-        return timestamp;
-    }
-
-    public void setTimestamp(Date timestamp) {
-        this.timestamp = timestamp;
-    }
-
-    /**
-     * The status metrics.
-     *
-     * @return
-     */
-    public Map<String, Long> getStatusMetrics() {
-        return statusMetrics;
-    }
-
-    public void setStatusMetrics(Map<String, Long> statusMetrics) {
-        this.statusMetrics = statusMetrics;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/DateTimeAdapter.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/DateTimeAdapter.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/DateTimeAdapter.java
deleted file mode 100644
index 46ba674..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/DateTimeAdapter.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.web.api.dto.util;
-
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Locale;
-import java.util.TimeZone;
-
-import javax.xml.bind.annotation.adapters.XmlAdapter;
-
-/**
- * XmlAdapter for (un)marshalling a date/time.
- */
-public class DateTimeAdapter extends XmlAdapter<String, Date> {
-
-    public static final String DEFAULT_DATE_TIME_FORMAT = "MM/dd/yyyy HH:mm:ss z";
-
-    @Override
-    public String marshal(Date date) throws Exception {
-        final SimpleDateFormat formatter = new SimpleDateFormat(DEFAULT_DATE_TIME_FORMAT, Locale.US);
-        formatter.setTimeZone(TimeZone.getDefault());
-        return formatter.format(date);
-    }
-
-    @Override
-    public Date unmarshal(String date) throws Exception {
-        final SimpleDateFormat parser = new SimpleDateFormat(DEFAULT_DATE_TIME_FORMAT, Locale.US);
-        parser.setTimeZone(TimeZone.getDefault());
-        return parser.parse(date);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/TimeAdapter.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/TimeAdapter.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/TimeAdapter.java
deleted file mode 100644
index 100d6c1..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/TimeAdapter.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.web.api.dto.util;
-
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Locale;
-import java.util.TimeZone;
-
-import javax.xml.bind.annotation.adapters.XmlAdapter;
-
-/**
- * XmlAdapter for (un)marshalling a time.
- */
-public class TimeAdapter extends XmlAdapter<String, Date> {
-
-    public static final String DEFAULT_TIME_FORMAT = "HH:mm:ss z";
-
-    @Override
-    public String marshal(Date date) throws Exception {
-        final SimpleDateFormat formatter = new SimpleDateFormat(DEFAULT_TIME_FORMAT, Locale.US);
-        formatter.setTimeZone(TimeZone.getDefault());
-        return formatter.format(date);
-    }
-
-    @Override
-    public Date unmarshal(String date) throws Exception {
-        final SimpleDateFormat parser = new SimpleDateFormat(DEFAULT_TIME_FORMAT, Locale.US);
-        parser.setTimeZone(TimeZone.getDefault());
-        return parser.parse(date);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/TimestampAdapter.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/TimestampAdapter.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/TimestampAdapter.java
deleted file mode 100644
index d182cf3..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/TimestampAdapter.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.web.api.dto.util;
-
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Locale;
-import java.util.TimeZone;
-
-import javax.xml.bind.annotation.adapters.XmlAdapter;
-
-/**
- * XmlAdapter for (un)marshalling a date/time.
- */
-public class TimestampAdapter extends XmlAdapter<String, Date> {
-
-    public static final String DEFAULT_DATE_TIME_FORMAT = "MM/dd/yyyy HH:mm:ss.SSS z";
-
-    @Override
-    public String marshal(Date date) throws Exception {
-        final SimpleDateFormat formatter = new SimpleDateFormat(DEFAULT_DATE_TIME_FORMAT, Locale.US);
-        formatter.setTimeZone(TimeZone.getDefault());
-        return formatter.format(date);
-    }
-
-    @Override
-    public Date unmarshal(String date) throws Exception {
-        final SimpleDateFormat parser = new SimpleDateFormat(DEFAULT_DATE_TIME_FORMAT, Locale.US);
-        parser.setTimeZone(TimeZone.getDefault());
-        return parser.parse(date);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/AboutEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/AboutEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/AboutEntity.java
deleted file mode 100644
index 36fc163..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/AboutEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.AboutDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a AboutDTO.
- */
-@XmlRootElement(name = "aboutEntity")
-public class AboutEntity extends Entity {
-
-    private AboutDTO about;
-
-    /**
-     * The AboutDTO that is being serialized.
-     *
-     * @return The AboutDTO object
-     */
-    public AboutDTO getAbout() {
-        return about;
-    }
-
-    public void setAbout(AboutDTO about) {
-        this.about = about;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ActionEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ActionEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ActionEntity.java
deleted file mode 100644
index 1b2c0b6..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ActionEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.action.ActionDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to an ActionDTO.
- */
-@XmlRootElement(name = "actionEntity")
-public class ActionEntity extends Entity {
-
-    private ActionDTO action;
-
-    /**
-     * The ActionDTO that is being serialized.
-     *
-     * @return The ActionDTO object
-     */
-    public ActionDTO getAction() {
-        return action;
-    }
-
-    public void setAction(ActionDTO action) {
-        this.action = action;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/AuthorityEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/AuthorityEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/AuthorityEntity.java
deleted file mode 100644
index eed1305..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/AuthorityEntity.java
+++ /dev/null
@@ -1,60 +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.api.entity;
-
-import java.util.Set;
-
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a response to the API. This particular entity holds a set of user
- * authorities.
- */
-@XmlRootElement(name = "authoritiesEntity")
-public class AuthorityEntity extends Entity {
-
-    private String userId;
-    private Set<String> authorities;
-
-    /**
-     * The current user id.
-     *
-     * @return
-     */
-    public String getUserId() {
-        return userId;
-    }
-
-    public void setUserId(String userId) {
-        this.userId = userId;
-    }
-
-    /**
-     * The set of authorities that are being serialized.
-     *
-     * @return
-     */
-    public Set<String> getAuthorities() {
-        return authorities;
-    }
-
-    public void setAuthorities(Set<String> authorities) {
-        this.authorities = authorities;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/BannerEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/BannerEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/BannerEntity.java
deleted file mode 100644
index 594707a..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/BannerEntity.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.web.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-
-import org.apache.nifi.web.api.dto.BannerDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a BannerDTO.
- */
-@XmlRootElement(name = "bannersEntity")
-public class BannerEntity extends Entity {
-
-    private BannerDTO banners;
-
-    /**
-     * The BannerDTO that is being serialized.
-     *
-     * @return The BannerDTO object
-     */
-    public BannerDTO getBanners() {
-        return banners;
-    }
-
-    public void setBanners(BannerDTO about) {
-        this.banners = about;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/BulletinBoardEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/BulletinBoardEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/BulletinBoardEntity.java
deleted file mode 100644
index 4287893..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/BulletinBoardEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.BulletinBoardDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a BulletinBoardDTO.
- */
-@XmlRootElement(name = "bulletinBoardEntity")
-public class BulletinBoardEntity extends Entity {
-
-    private BulletinBoardDTO messages;
-
-    /**
-     * The BulletinBoardDTO that is being serialized.
-     *
-     * @return The BulletinBoardDTO object
-     */
-    public BulletinBoardDTO getBulletinBoard() {
-        return messages;
-    }
-
-    public void setBulletinBoard(BulletinBoardDTO messages) {
-        this.messages = messages;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterConnectionStatusEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterConnectionStatusEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterConnectionStatusEntity.java
deleted file mode 100644
index 732a4b7..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterConnectionStatusEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.status.ClusterConnectionStatusDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a ClusterConnectionStatusDTO.
- */
-@XmlRootElement(name = "clusterConnectionStatusEntity")
-public class ClusterConnectionStatusEntity extends Entity {
-
-    private ClusterConnectionStatusDTO clusterConnectionStatus;
-
-    /**
-     * The ClusterConnectionStatusDTO that is being serialized.
-     *
-     * @return The ClusterConnectionStatusDTO object
-     */
-    public ClusterConnectionStatusDTO getClusterConnectionStatus() {
-        return clusterConnectionStatus;
-    }
-
-    public void setClusterConnectionStatus(ClusterConnectionStatusDTO clusterConnectionStatus) {
-        this.clusterConnectionStatus = clusterConnectionStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterEntity.java
deleted file mode 100644
index 8306b0f..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.ClusterDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a ClusterDTO.
- */
-@XmlRootElement(name = "clusterEntity")
-public class ClusterEntity extends Entity {
-
-    private ClusterDTO cluster;
-
-    /**
-     * The ClusterDTO that is being serialized.
-     *
-     * @return The ClusterDTO object
-     */
-    public ClusterDTO getCluster() {
-        return cluster;
-    }
-
-    public void setCluster(ClusterDTO cluster) {
-        this.cluster = cluster;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterPortStatusEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterPortStatusEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterPortStatusEntity.java
deleted file mode 100644
index 16a2497..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterPortStatusEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.status.ClusterPortStatusDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a ClusterPortStatusDTO.
- */
-@XmlRootElement(name = "clusterPortStatusEntity")
-public class ClusterPortStatusEntity extends Entity {
-
-    private ClusterPortStatusDTO clusterPortStatus;
-
-    /**
-     * The ClusterPortStatusDTO that is being serialized.
-     *
-     * @return The ClusterPortStatusDTO object
-     */
-    public ClusterPortStatusDTO getClusterPortStatus() {
-        return clusterPortStatus;
-    }
-
-    public void setClusterPortStatus(ClusterPortStatusDTO clusterPortStatus) {
-        this.clusterPortStatus = clusterPortStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessorStatusEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessorStatusEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessorStatusEntity.java
deleted file mode 100644
index 68e5c5c..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessorStatusEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.status.ClusterProcessorStatusDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a ClusterProcessorStatusDTO.
- */
-@XmlRootElement(name = "clusterProcessorStatusEntity")
-public class ClusterProcessorStatusEntity extends Entity {
-
-    private ClusterProcessorStatusDTO clusterProcessorStatus;
-
-    /**
-     * The ClusterProcessorStatusDTO that is being serialized.
-     *
-     * @return The ClusterProcessorStatusDTO object
-     */
-    public ClusterProcessorStatusDTO getClusterProcessorStatus() {
-        return clusterProcessorStatus;
-    }
-
-    public void setClusterProcessorStatus(ClusterProcessorStatusDTO clusterProcessorStatus) {
-        this.clusterProcessorStatus = clusterProcessorStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterRemoteProcessGroupStatusEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterRemoteProcessGroupStatusEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterRemoteProcessGroupStatusEntity.java
deleted file mode 100644
index 8a3da7c..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterRemoteProcessGroupStatusEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.status.ClusterRemoteProcessGroupStatusDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a ClusterRemoteProcessGroupStatusDTO.
- */
-@XmlRootElement(name = "clusterRemoteProcessGroupStatusEntity")
-public class ClusterRemoteProcessGroupStatusEntity extends Entity {
-
-    private ClusterRemoteProcessGroupStatusDTO clusterRemoteProcessGroupStatus;
-
-    /**
-     * The ClusterRemoteProcessGroupStatusDTO that is being serialized.
-     *
-     * @return The ClusterRemoteProcessGroupStatusDTO object
-     */
-    public ClusterRemoteProcessGroupStatusDTO getClusterRemoteProcessGroupStatus() {
-        return clusterRemoteProcessGroupStatus;
-    }
-
-    public void setClusterRemoteProcessGroupStatus(ClusterRemoteProcessGroupStatusDTO clusterRemoteProcessGroupStatus) {
-        this.clusterRemoteProcessGroupStatus = clusterRemoteProcessGroupStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterSearchResultsEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterSearchResultsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterSearchResultsEntity.java
deleted file mode 100644
index 546bd8b..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterSearchResultsEntity.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.web.api.entity;
-
-import java.util.List;
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.search.NodeSearchResultDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to NodeSearchResultDTOs.
- */
-@XmlRootElement(name = "clusterSearchResultsEntity")
-public class ClusterSearchResultsEntity {
-
-    private List<NodeSearchResultDTO> nodeResults;
-
-    /**
-     * The node search results.
-     *
-     * @return
-     */
-    public List<NodeSearchResultDTO> getNodeResults() {
-        return nodeResults;
-    }
-
-    public void setNodeResults(List<NodeSearchResultDTO> nodeResults) {
-        this.nodeResults = nodeResults;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusEntity.java
deleted file mode 100644
index d22d432..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.status.ClusterStatusDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a ClusterStatusDTO.
- */
-@XmlRootElement(name = "clusterStatusEntity")
-public class ClusterStatusEntity extends Entity {
-
-    private ClusterStatusDTO clusterStatus;
-
-    /**
-     * The ClusterStatusDTO that is being serialized.
-     *
-     * @return The ClusterStatusDTO object
-     */
-    public ClusterStatusDTO getClusterStatus() {
-        return clusterStatus;
-    }
-
-    public void setClusterStatus(ClusterStatusDTO clusterStatus) {
-        this.clusterStatus = clusterStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusHistoryEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusHistoryEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusHistoryEntity.java
deleted file mode 100644
index 9ed69fd..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusHistoryEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.status.ClusterStatusHistoryDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a ClusterStatusHistoryDTO.
- */
-@XmlRootElement(name = "clusterStatusHistoryEntity")
-public class ClusterStatusHistoryEntity extends Entity {
-
-    private ClusterStatusHistoryDTO clusterStatusHistory;
-
-    /**
-     * The ClusterStatusHistoryDTO that is being serialized.
-     *
-     * @return The ClusterStatusHistoryDTO object
-     */
-    public ClusterStatusHistoryDTO getClusterStatusHistory() {
-        return clusterStatusHistory;
-    }
-
-    public void setClusterStatusHistory(ClusterStatusHistoryDTO clusterStatusHistory) {
-        this.clusterStatusHistory = clusterStatusHistory;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionEntity.java
deleted file mode 100644
index 69ae937..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.ConnectionDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a response to the API. This particular entity holds a reference to a
- * ConnectionDTO.
- */
-@XmlRootElement(name = "connectionEntity")
-public class ConnectionEntity extends Entity {
-
-    private ConnectionDTO connection;
-
-    /**
-     * The RelationshipDTO that is being serialized.
-     *
-     * @return
-     */
-    public ConnectionDTO getConnection() {
-        return connection;
-    }
-
-    public void setConnection(ConnectionDTO connection) {
-        this.connection = connection;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionsEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionsEntity.java
deleted file mode 100644
index c69c69a..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionsEntity.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.web.api.entity;
-
-import java.util.Set;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.ConnectionDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a response to the API. This particular entity holds a reference to a list of
- * ConnectionDTOs.
- */
-@XmlRootElement(name = "connectionsEntity")
-public class ConnectionsEntity extends Entity {
-
-    private Set<ConnectionDTO> connections;
-
-    /**
-     * The list of ConnectionDTOs that are being serialized.
-     *
-     * @return
-     */
-    public Set<ConnectionDTO> getConnections() {
-        return connections;
-    }
-
-    public void setConnections(Set<ConnectionDTO> connections) {
-        this.connections = connections;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerConfigurationEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerConfigurationEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerConfigurationEntity.java
deleted file mode 100644
index e13903b..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerConfigurationEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.ControllerConfigurationDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a ControllerConfigurationDTO.
- */
-@XmlRootElement(name = "controllerConfigurationEntity")
-public class ControllerConfigurationEntity extends Entity {
-
-    private ControllerConfigurationDTO config;
-
-    /**
-     * The ControllerConfigurationDTO that is being serialized.
-     *
-     * @return The ControllerConfigurationDTO object
-     */
-    public ControllerConfigurationDTO getConfig() {
-        return config;
-    }
-
-    public void setConfig(ControllerConfigurationDTO config) {
-        this.config = config;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerEntity.java
deleted file mode 100644
index 62d0409..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.ControllerDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a ControllerDTO.
- */
-@XmlRootElement(name = "controllerEntity")
-public class ControllerEntity extends Entity {
-
-    private ControllerDTO controller;
-
-    /**
-     * The ControllerDTO that is being serialized.
-     *
-     * @return The ControllerDTO object
-     */
-    public ControllerDTO getController() {
-        return controller;
-    }
-
-    public void setController(ControllerDTO controller) {
-        this.controller = controller;
-    }
-
-}


[31/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.py
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.py
deleted file mode 100644
index ed5408a..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.py
+++ /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.
-import re
-
-class RoutingReader(ReaderScript):
-    A = Relationship.Builder().name("a").description("some good stuff").build()
-    B = Relationship.Builder().name("b").description("some other stuff").build()
-    C = Relationship.Builder().name("c").description("some bad stuff").build()
-    
-    def getRelationships(self):
-        return [self.A,self.B,self.C]
-  
-    def getExceptionRoute(self):
-        return self.C
-  
-    def route( self, input ):
-        for line in FileUtil.wrap(input):
-            if re.match("^bad", line, re.IGNORECASE):
-                return self.B
-            if re.match("^sed", line):
-                raise RuntimeError("That's no good!")
-
-        return self.A
-
-instance = RoutingReader()

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.rb
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.rb
deleted file mode 100644
index 38249ae..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.rb
+++ /dev/null
@@ -1,39 +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.
-class SimpleJRubyReader < ReaderScript
-
-    @@a = Relationship::Builder.new().name("a").description("some good stuff").build()
-    @@b = Relationship::Builder.new().name("b").description("some bad stuff").build()
-    @@c = Relationship::Builder.new().name("c").description("some other stuff").build()
-  
-  def getRelationships
-    return [@@a, @@b, @@c]
-  end
-  
-  def getExceptionRoute
-    @@c
-  end
-  
-  def route( input )
-    input.to_io.each_line do |line|
-      return @@b if line.match /^bad/i
-      raise "That's no good!" if line.match /^sed/i
-    end
-
-    @@a
-	end
-end
-
-SimpleJRubyReader.new

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.js
deleted file mode 100644
index 72bb80e..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.js
+++ /dev/null
@@ -1,45 +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.
- */
-with (Scripting) {
-
-    var instance = new ConverterScript({
-        convert: function (input) {
-            var buffReader = new java.io.BufferedReader(new java.io.InputStreamReader(input));
-            instance.createFlowFile("firstLine", Script.FAIL_RELATIONSHIP, function (output) {
-                var out = new java.io.BufferedWriter(new java.io.OutputStreamWriter(output));
-                var firstLine = buffReader.readLine();
-                out.write(firstLine, 0, firstLine.length());
-                out.flush();
-                out.close();
-            });
-
-            instance.createFlowFile("otherLines", Script.SUCCESS_RELATIONSHIP, function (output) {
-                var out = new java.io.BufferedWriter(new java.io.OutputStreamWriter(output));
-                var line = buffReader.readLine();
-                while (line != null) {
-                    out.write(line, 0, line.length());
-                    out.newLine();
-                    line = buffReader.readLine();
-                }
-                out.flush();
-                out.close();
-            });
-        }
-
-    });
-    logger.debug("Processor props" + properties)
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.py
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.py
deleted file mode 100644
index d52368e..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.py
+++ /dev/null
@@ -1,60 +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.
-from org.python.core.io import TextIOWrapper,BufferedReader,BufferedWriter,StreamIO
-from org.apache.nifi.scripting import OutputStreamHandler
-
-class WriteFirstLine(OutputStreamHandler):
-    def __init__(self, wrappedIn):
-        self.wrappedIn = wrappedIn
-    
-    def write(self, output):
-        streamOut = StreamIO(output, False)
-        bufWrtr = BufferedWriter(streamOut, 8192)
-        wrappedOut = TextIOWrapper(bufWrtr)
-        wrappedOut.write(self.wrappedIn.readline(8192))
-        wrappedOut.flush()
-        wrappedOut.close()
-    
-class WriteOtherLines(OutputStreamHandler):
-    def __init__(self, wrappedIn):
-        self.wrappedIn = wrappedIn
-        
-    def write(self, output):
-        streamOut = StreamIO(output, False)
-        bufWrtr = BufferedWriter(streamOut, 8192)
-        wrappedOut = TextIOWrapper(bufWrtr)
-        line = self.wrappedIn.readline(8192)
-        while line != '':
-            wrappedOut.write(line)
-            line = self.wrappedIn.readline(8192)
-        wrappedOut.flush()
-        wrappedOut.close()
-
-class SimpleConverter(ConverterScript):
-
-    def convert(self, input):
-        streamIn = StreamIO(input, False)
-        bufRdr = BufferedReader(streamIn, 8192)
-        wrappedIn = TextIOWrapper(bufRdr)
-        
-        writeFirstLine = WriteFirstLine(wrappedIn)
-        self.createFlowFile("firstLine", self.FAIL_RELATIONSHIP, writeFirstLine)
-
-        writeOtherLines = WriteOtherLines(wrappedIn)                
-        self.createFlowFile("otherLines", self.SUCCESS_RELATIONSHIP, writeOtherLines)     
-        
-instance = SimpleConverter()
-        
-        
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.rb
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.rb
deleted file mode 100644
index f333294..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.rb
+++ /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.
-java_import 'org.apache.nifi.scripting.OutputStreamHandler'
-class SimpleConverter < ConverterScript
-  field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP, :logger, :attributes
-  
-  def convert(input)
-    in_io = input.to_io
-    createFlowFile("firstLine", FAIL_RELATIONSHIP, OutputStreamHandler.impl do |method, out|
-	    out_io = out.to_io
-		out_io << in_io.readline.to_java_bytes
-	    out_io.close
- 	    logger.debug("Wrote data to failure...this message logged with logger from super class")
-      end)
-	  
-    createFlowFile("otherLines", SUCCESS_RELATIONSHIP, OutputStreamHandler.impl do |method, out|
-		out_io = out.to_io
-		in_io.each_line { |line|
-		  out_io << line
-		}
-		out_io.close
-		logger.debug("Wrote data to success...this message logged with logger from super class")
-      end)
-	in_io.close
-  end
-   
-end
-
-$logger.debug("Creating SimpleConverter...this message logged with logger from shared variables")
-SimpleConverter.new
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.js
deleted file mode 100644
index 1e055b7..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.js
+++ /dev/null
@@ -1,26 +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.
- */
-with (Scripting) {
-    var instance = new WriterScript({
-        process: function (input, output) {
-            var str = IOUtils.toString(input);
-            IOUtils.write(str.split("\n").pop(), output);
-            output.flush();
-        }
-    });
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.py
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.py
deleted file mode 100644
index c563b66..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.py
+++ /dev/null
@@ -1,22 +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.
-class SimpleWriter(WriterScript):
-    def process( self, input, output ):
-        last = FileUtil.wrap(input).readlines()[-1]
-        writer = FileUtil.wrap(output)
-        writer.write(last)
-        writer.close()
-
-instance = SimpleWriter()

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.rb
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.rb
deleted file mode 100644
index 5be4553..0000000
--- a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.rb
+++ /dev/null
@@ -1,32 +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.
-class SimpleJRubyRunner < WriterScript
-  def process( input, output )
-    in_io = input.to_io
-    out_io = output.to_io 
-    
-    last = nil
-    in_io.each_line do |line|
-       last = line
-    end
-
-    out_io << last unless last.nil?
-    
-    in_io.close
-    out_io.close
-	end
-end
-
-SimpleJRubyRunner.new

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/nar/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/nar/pom.xml b/nar-bundles/execute-script-bundle/nar/pom.xml
deleted file mode 100644
index 1cc28dc..0000000
--- a/nar-bundles/execute-script-bundle/nar/pom.xml
+++ /dev/null
@@ -1,36 +0,0 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-    <!--
-      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.
-    -->
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>execute-script-bundle</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-
-    <artifactId>execute-script-nar</artifactId>
-    <name>Execute Script NAR</name>
-    <packaging>nar</packaging>
-    <description>NiFi Script Running NAR</description>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>execute-script-processors</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/execute-script-bundle/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/pom.xml b/nar-bundles/execute-script-bundle/pom.xml
deleted file mode 100644
index b3abb36..0000000
--- a/nar-bundles/execute-script-bundle/pom.xml
+++ /dev/null
@@ -1,81 +0,0 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <!--
-      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.
-    -->
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nar-container-common</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-
-    <artifactId>execute-script-bundle</artifactId>
-    <version>0.0.1-SNAPSHOT</version>
-
-    <name>Execute Script Bundle</name>
-    <packaging>pom</packaging>
-
-    <dependencyManagement>
-        <dependencies>
-            <dependency>
-                <groupId>org.apache.nifi</groupId>
-                <artifactId>nifi-processor-utils</artifactId>
-                <version>0.0.1-SNAPSHOT</version>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.nifi</groupId>
-                <artifactId>nifi-stream-utils</artifactId>
-                <version>0.0.1-SNAPSHOT</version>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.nifi</groupId>
-                <artifactId>nifi-utils</artifactId>
-                <version>0.0.1-SNAPSHOT</version>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.nifi</groupId>
-                <artifactId>nifi-core-flowfile-attributes</artifactId>
-                <version>0.0.1-SNAPSHOT</version>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.nifi</groupId>
-                <artifactId>nifi-mock</artifactId>
-                <version>0.0.1-SNAPSHOT</version>
-                <scope>test</scope>
-            </dependency>
-
-            <dependency>
-                <groupId>org.jruby</groupId>
-                <artifactId>jruby</artifactId>
-                <version>1.7.16.1</version>
-            </dependency>
-	        
-            <dependency>
-                <groupId>org.python</groupId>
-                <artifactId>jython-standalone</artifactId>
-                <version>2.7-b3</version>
-            </dependency>
-            <dependency>
-                <groupId>commons-io</groupId>
-                <artifactId>commons-io</artifactId>
-                <version>2.4</version>
-            </dependency>
-        </dependencies>
-    </dependencyManagement>
-
-    <modules>
-        <module>execute-script-processors</module>
-        <module>nar</module>
-    </modules>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/.gitignore
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/.gitignore b/nar-bundles/framework-bundle/framework/administration/.gitignore
deleted file mode 100755
index ea8c4bf..0000000
--- a/nar-bundles/framework-bundle/framework/administration/.gitignore
+++ /dev/null
@@ -1 +0,0 @@
-/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/pom.xml b/nar-bundles/framework-bundle/framework/administration/pom.xml
deleted file mode 100644
index b5dd171..0000000
--- a/nar-bundles/framework-bundle/framework/administration/pom.xml
+++ /dev/null
@@ -1,116 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-framework-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-
-    <artifactId>nifi-administration</artifactId>
-    <version>0.0.1-SNAPSHOT</version>
-    <name>NiFi Administration</name>
-    <build>
-        <resources>
-            <resource>
-                <directory>src/main/resources</directory>
-            </resource>
-            <resource>
-                <directory>src/main/xsd</directory>
-            </resource>
-        </resources>
-        <plugins>
-            <plugin>
-                <groupId>org.codehaus.mojo</groupId>
-                <artifactId>jaxb2-maven-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <id>current</id>
-                        <goals>
-                            <goal>xjc</goal>
-                        </goals>
-                        <configuration>
-                            <packageName>org.apache.nifi.authorization.generated</packageName>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-        </plugins>
-    </build>
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-user-actions</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>core-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-nar</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-properties</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-security-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.h2database</groupId>
-            <artifactId>h2</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-beans</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-context</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-core</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-aop</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.aspectj</groupId>
-            <artifactId>aspectjweaver</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.springframework.security</groupId>
-            <artifactId>spring-security-core</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-collections4</artifactId>
-        </dependency>
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/AuditDataSourceFactoryBean.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/AuditDataSourceFactoryBean.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/AuditDataSourceFactoryBean.java
deleted file mode 100644
index aeb2755..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/AuditDataSourceFactoryBean.java
+++ /dev/null
@@ -1,222 +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.admin;
-
-import java.io.File;
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import org.apache.commons.lang3.StringUtils;
-import org.h2.jdbcx.JdbcConnectionPool;
-import org.apache.nifi.util.NiFiProperties;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.beans.factory.FactoryBean;
-
-/**
- *
- */
-public class AuditDataSourceFactoryBean implements FactoryBean {
-
-    private static final Logger logger = LoggerFactory.getLogger(AuditDataSourceFactoryBean.class);
-    private static final String NF_USERNAME_PASSWORD = "nf";
-    private static final int MAX_CONNECTIONS = 5;
-
-    // database file name
-    private static final String AUDIT_DATABASE_FILE_NAME = "nifi-audit";
-
-    // ------------
-    // action table
-    // ------------
-    private static final String CREATE_ACTION_TABLE = "CREATE TABLE ACTION ("
-            + "ID INT NOT NULL PRIMARY KEY AUTO_INCREMENT, "
-            + "USER_DN VARCHAR2(255) NOT NULL, "
-            + "USER_NAME VARCHAR2(100) NOT NULL, "
-            + "SOURCE_ID VARCHAR2(100) NOT NULL, "
-            + "SOURCE_NAME VARCHAR2(1000) NOT NULL, "
-            + "SOURCE_TYPE VARCHAR2(1000) NOT NULL, "
-            + "OPERATION VARCHAR2(50) NOT NULL, "
-            + "ACTION_TIMESTAMP TIMESTAMP NOT NULL "
-            + ")";
-
-    // -----------------
-    // component details
-    // -----------------
-    private static final String CREATE_PROCESSOR_DETAILS_TABLE = "CREATE TABLE PROCESSOR_DETAILS ("
-            + "ACTION_ID INT NOT NULL PRIMARY KEY, "
-            + "TYPE VARCHAR2(1000) NOT NULL, "
-            + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)"
-            + ")";
-
-    private static final String CREATE_REMOTE_PROCESS_GROUP_DETAILS_TABLE = "CREATE TABLE REMOTE_PROCESS_GROUP_DETAILS ("
-            + "ACTION_ID INT NOT NULL PRIMARY KEY, "
-            + "URI VARCHAR2(2500) NOT NULL, "
-            + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)"
-            + ")";
-
-    // --------------
-    // action details
-    // --------------
-    private static final String CREATE_MOVE_DETAILS_TABLE = "CREATE TABLE MOVE_DETAILS ("
-            + "ACTION_ID INT NOT NULL PRIMARY KEY, "
-            + "GROUP_ID VARCHAR2(100) NOT NULL, "
-            + "GROUP_NAME VARCHAR2(1000) NOT NULL, "
-            + "PREVIOUS_GROUP_ID VARCHAR2(100) NOT NULL, "
-            + "PREVIOUS_GROUP_NAME VARCHAR2(1000) NOT NULL, "
-            + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)"
-            + ")";
-
-    private static final String CREATE_CONFIGURE_DETAILS_TABLE = "CREATE TABLE CONFIGURE_DETAILS ("
-            + "ACTION_ID INT NOT NULL PRIMARY KEY, "
-            + "NAME VARCHAR2(1000) NOT NULL, "
-            + "VALUE VARCHAR2(5000), "
-            + "PREVIOUS_VALUE VARCHAR2(5000), "
-            + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)"
-            + ")";
-
-    private static final String CREATE_CONNECT_DETAILS_TABLE = "CREATE TABLE CONNECT_DETAILS ("
-            + "ACTION_ID INT NOT NULL PRIMARY KEY, "
-            + "SOURCE_ID VARCHAR2(100) NOT NULL, "
-            + "SOURCE_NAME VARCHAR2(1000), "
-            + "SOURCE_TYPE VARCHAR2(1000) NOT NULL, "
-            + "RELATIONSHIP VARCHAR2(1000), "
-            + "DESTINATION_ID VARCHAR2(100) NOT NULL, "
-            + "DESTINATION_NAME VARCHAR2(1000), "
-            + "DESTINATION_TYPE VARCHAR2(1000) NOT NULL, "
-            + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)"
-            + ")";
-
-    private static final String CREATE_PURGE_DETAILS_TABLE = "CREATE TABLE PURGE_DETAILS ("
-            + "ACTION_ID INT NOT NULL PRIMARY KEY, "
-            + "END_DATE TIMESTAMP NOT NULL, "
-            + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)"
-            + ")";
-
-    private JdbcConnectionPool connectionPool;
-
-    private NiFiProperties properties;
-
-    @Override
-    public Object getObject() throws Exception {
-        if (connectionPool == null) {
-
-            // locate the repository directory
-            String repositoryDirectoryPath = properties.getProperty(NiFiProperties.REPOSITORY_DATABASE_DIRECTORY);
-
-            // ensure the repository directory is specified
-            if (repositoryDirectoryPath == null) {
-                throw new NullPointerException("Database directory must be specified.");
-            }
-
-            // create a handle to the repository directory
-            File repositoryDirectory = new File(repositoryDirectoryPath);
-
-            // get a handle to the database file
-            File databaseFile = new File(repositoryDirectory, AUDIT_DATABASE_FILE_NAME);
-
-            // format the database url
-            String databaseUrl = "jdbc:h2:" + databaseFile + ";AUTOCOMMIT=OFF;DB_CLOSE_ON_EXIT=FALSE;LOCK_MODE=3";
-            String databaseUrlAppend = properties.getProperty(NiFiProperties.H2_URL_APPEND);
-            if (StringUtils.isNotBlank(databaseUrlAppend)) {
-                databaseUrl += databaseUrlAppend;
-            }
-
-            // create the pool
-            connectionPool = JdbcConnectionPool.create(databaseUrl, NF_USERNAME_PASSWORD, NF_USERNAME_PASSWORD);
-            connectionPool.setMaxConnections(MAX_CONNECTIONS);
-
-            Connection connection = null;
-            ResultSet rs = null;
-            Statement statement = null;
-            try {
-                // get a connection
-                connection = connectionPool.getConnection();
-                connection.setAutoCommit(false);
-
-                // determine if the tables need to be created
-                rs = connection.getMetaData().getTables(null, null, "ACTION", null);
-                if (!rs.next()) {
-                    logger.info("Database not built for repository: " + databaseUrl + ".  Building now...");
-                    RepositoryUtils.closeQuietly(rs);
-
-                    // create a statement for initializing the database
-                    statement = connection.createStatement();
-
-                    // action table
-                    statement.execute(CREATE_ACTION_TABLE);
-
-                    // component details
-                    statement.execute(CREATE_PROCESSOR_DETAILS_TABLE);
-                    statement.execute(CREATE_REMOTE_PROCESS_GROUP_DETAILS_TABLE);
-
-                    // action details
-                    statement.execute(CREATE_MOVE_DETAILS_TABLE);
-                    statement.execute(CREATE_CONFIGURE_DETAILS_TABLE);
-                    statement.execute(CREATE_CONNECT_DETAILS_TABLE);
-                    statement.execute(CREATE_PURGE_DETAILS_TABLE);
-                } else {
-                    logger.info("Existing database found and connected to at: " + databaseUrl);
-                }
-
-                // commit any changes
-                connection.commit();
-            } catch (SQLException sqle) {
-                RepositoryUtils.rollback(connection, logger);
-                throw sqle;
-            } finally {
-                RepositoryUtils.closeQuietly(rs);
-                RepositoryUtils.closeQuietly(statement);
-                RepositoryUtils.closeQuietly(connection);
-            }
-        }
-
-        return connectionPool;
-    }
-
-    @Override
-    public Class getObjectType() {
-        return JdbcConnectionPool.class;
-    }
-
-    @Override
-    public boolean isSingleton() {
-        return true;
-    }
-
-    public void setProperties(NiFiProperties properties) {
-        this.properties = properties;
-    }
-
-    /**
-     * Disposes resources.
-     */
-    public void shutdown() {
-
-        // shutdown the connection pool
-        if (connectionPool != null) {
-            try {
-                connectionPool.dispose();
-            } catch (Exception e) {
-                logger.warn("Unable to dispose of connection pool: " + e.getMessage());
-                if (logger.isDebugEnabled()) {
-                    logger.warn(StringUtils.EMPTY, e);
-                }
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/RepositoryUtils.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/RepositoryUtils.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/RepositoryUtils.java
deleted file mode 100644
index b95388b..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/RepositoryUtils.java
+++ /dev/null
@@ -1,91 +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.admin;
-
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import org.slf4j.Logger;
-
-/**
- * A utility class for useful methods dealing with the repository
- *
- * @author unattributed
- */
-public class RepositoryUtils {
-
-    public static void rollback(final Connection conn, final Logger logger) {
-        try {
-            if (null != conn) {
-                conn.rollback();
-            }
-        } catch (final SQLException sqe) {
-            logger.warn("The following problem occurred while trying to rollback " + conn + ": " + sqe.getLocalizedMessage());
-            if (logger.isDebugEnabled()) {
-                logger.debug("", sqe);
-            }
-        }
-    }
-
-    /**
-     * Closes the given statement quietly - no logging, no exceptions
-     *
-     * @param statement
-     */
-    public static void closeQuietly(final Statement statement) {
-
-        if (null != statement) {
-            try {
-                statement.close();
-            } catch (final SQLException se) { /*IGNORE*/
-
-            }
-        }
-    }
-
-    /**
-     * Closes the given result set quietly - no logging, no exceptions
-     *
-     * @param resultSet
-     */
-    public static void closeQuietly(final ResultSet resultSet) {
-        if (null != resultSet) {
-            try {
-                resultSet.close();
-            } catch (final SQLException se) {/*IGNORE*/
-
-            }
-        }
-    }
-
-    /**
-     * Closes the given connection quietly - no logging, no exceptions
-     *
-     * @param conn
-     */
-    public static void closeQuietly(final Connection conn) {
-        if (null != conn) {
-            try {
-                conn.close();
-            } catch (final SQLException se) {/*IGNORE*/
-
-            }
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/UserDataSourceFactoryBean.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/UserDataSourceFactoryBean.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/UserDataSourceFactoryBean.java
deleted file mode 100644
index 1f64f6e..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/UserDataSourceFactoryBean.java
+++ /dev/null
@@ -1,247 +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.admin;
-
-import java.io.File;
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.UUID;
-import org.apache.commons.lang3.StringUtils;
-import org.h2.jdbcx.JdbcConnectionPool;
-import org.apache.nifi.user.NiFiUser;
-import org.apache.nifi.util.NiFiProperties;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.beans.factory.FactoryBean;
-
-/**
- *
- */
-public class UserDataSourceFactoryBean implements FactoryBean {
-
-    private static final Logger logger = LoggerFactory.getLogger(UserDataSourceFactoryBean.class);
-    private static final String NF_USERNAME_PASSWORD = "nf";
-    private static final int MAX_CONNECTIONS = 5;
-
-    // database file name
-    private static final String AUDIT_DATABASE_FILE_NAME = "nifi-users";
-
-    private static final String CREATE_USER_TABLE = "CREATE TABLE USER ("
-            + "ID VARCHAR2(100) NOT NULL PRIMARY KEY, "
-            + "DN VARCHAR2(255) NOT NULL UNIQUE, "
-            + "USER_NAME VARCHAR2(100) NOT NULL, "
-            + "USER_GROUP VARCHAR2(100), "
-            + "CREATION TIMESTAMP NOT NULL, "
-            + "LAST_ACCESSED TIMESTAMP, "
-            + "LAST_VERIFIED TIMESTAMP, "
-            + "JUSTIFICATION VARCHAR2(500) NOT NULL, "
-            + "STATUS VARCHAR2(10) NOT NULL"
-            + ")";
-
-    private static final String CREATE_AUTHORITY_TABLE = "CREATE TABLE AUTHORITY ("
-            + "ID INT NOT NULL PRIMARY KEY AUTO_INCREMENT, "
-            + "USER_ID VARCHAR2(100) NOT NULL, "
-            + "ROLE VARCHAR2(50) NOT NULL, "
-            + "FOREIGN KEY (USER_ID) REFERENCES USER (ID), "
-            + "CONSTRAINT USER_ROLE_UNIQUE_CONSTRAINT UNIQUE (USER_ID, ROLE)"
-            + ")";
-
-    private static final String INSERT_ANONYMOUS_USER = "INSERT INTO USER ("
-            + "ID, DN, USER_NAME, CREATION, LAST_VERIFIED, JUSTIFICATION, STATUS"
-            + ") VALUES ("
-            + "'" + UUID.randomUUID().toString() + "', "
-            + "'" + NiFiUser.ANONYMOUS_USER_DN + "', "
-            + "'" + NiFiUser.ANONYMOUS_USER_DN + "', "
-            + "NOW(), "
-            + "NOW(), "
-            + "'Anonymous user needs no justification', "
-            + "'ACTIVE'"
-            + ")";
-
-    private static final String INSERT_ANONYMOUS_MONITOR_AUTHORITY = "INSERT INTO AUTHORITY ("
-            + "USER_ID, ROLE"
-            + ") VALUES ("
-            + "(SELECT ID FROM USER WHERE DN = '" + NiFiUser.ANONYMOUS_USER_DN + "'), "
-            + "'ROLE_MONITOR'"
-            + ")";
-
-    private static final String INSERT_ANONYMOUS_DFM_AUTHORITY = "INSERT INTO AUTHORITY ("
-            + "USER_ID, ROLE"
-            + ") VALUES ("
-            + "(SELECT ID FROM USER WHERE DN = '" + NiFiUser.ANONYMOUS_USER_DN + "'), "
-            + "'ROLE_DFM'"
-            + ")";
-
-    private static final String INSERT_ANONYMOUS_ADMIN_AUTHORITY = "INSERT INTO AUTHORITY ("
-            + "USER_ID, ROLE"
-            + ") VALUES ("
-            + "(SELECT ID FROM USER WHERE DN = '" + NiFiUser.ANONYMOUS_USER_DN + "'), "
-            + "'ROLE_ADMIN'"
-            + ")";
-
-    private static final String INSERT_ANONYMOUS_NIFI_AUTHORITY = "INSERT INTO AUTHORITY ("
-            + "USER_ID, ROLE"
-            + ") VALUES ("
-            + "(SELECT ID FROM USER WHERE DN = '" + NiFiUser.ANONYMOUS_USER_DN + "'), "
-            + "'ROLE_NIFI'"
-            + ")";
-
-    private static final String INSERT_ANONYMOUS_PROVENANCE_AUTHORITY = "INSERT INTO AUTHORITY ("
-            + "USER_ID, ROLE"
-            + ") VALUES ("
-            + "(SELECT ID FROM USER WHERE DN = '" + NiFiUser.ANONYMOUS_USER_DN + "'), "
-            + "'ROLE_PROVENANCE'"
-            + ")";
-
-    private static final String SELECT_ANONYMOUS_PROVENANCE_AUTHORITY = "SELECT * FROM AUTHORITY "
-            + "WHERE "
-            + "USER_ID = (SELECT ID FROM USER WHERE DN = '" + NiFiUser.ANONYMOUS_USER_DN + "') "
-            + "AND "
-            + "ROLE = 'ROLE_PROVENANCE'";
-
-    private JdbcConnectionPool connectionPool;
-
-    private NiFiProperties properties;
-
-    @Override
-    public Object getObject() throws Exception {
-        if (connectionPool == null) {
-
-            // locate the repository directory
-            String repositoryDirectoryPath = properties.getProperty(NiFiProperties.REPOSITORY_DATABASE_DIRECTORY);
-
-            // ensure the repository directory is specified
-            if (repositoryDirectoryPath == null) {
-                throw new NullPointerException("Database directory must be specified.");
-            }
-
-            // create a handle to the repository directory
-            File repositoryDirectory = new File(repositoryDirectoryPath);
-
-            // create a handle to the database directory and file
-            File databaseFile = new File(repositoryDirectory, AUDIT_DATABASE_FILE_NAME);
-            String databaseUrl = getDatabaseUrl(databaseFile);
-
-            // create the pool
-            connectionPool = JdbcConnectionPool.create(databaseUrl, NF_USERNAME_PASSWORD, NF_USERNAME_PASSWORD);
-            connectionPool.setMaxConnections(MAX_CONNECTIONS);
-
-            Connection connection = null;
-            ResultSet rs = null;
-            Statement statement = null;
-            try {
-                // get a connection
-                connection = connectionPool.getConnection();
-                connection.setAutoCommit(false);
-
-                // create a statement for creating/updating the database
-                statement = connection.createStatement();
-
-                // determine if the tables need to be created
-                rs = connection.getMetaData().getTables(null, null, "USER", null);
-                if (!rs.next()) {
-                    logger.info("Database not built for repository: " + databaseUrl + ".  Building now...");
-
-                    // create the tables
-                    statement.execute(CREATE_USER_TABLE);
-                    statement.execute(CREATE_AUTHORITY_TABLE);
-
-                    // seed the anonymous user
-                    statement.execute(INSERT_ANONYMOUS_USER);
-                    statement.execute(INSERT_ANONYMOUS_MONITOR_AUTHORITY);
-                    statement.execute(INSERT_ANONYMOUS_DFM_AUTHORITY);
-                    statement.execute(INSERT_ANONYMOUS_ADMIN_AUTHORITY);
-                    statement.execute(INSERT_ANONYMOUS_NIFI_AUTHORITY);
-                } else {
-                    logger.info("Existing database found and connected to at: " + databaseUrl);
-                }
-
-                // close the previous result set
-                RepositoryUtils.closeQuietly(rs);
-
-                // merge in the provenance role to handle existing databases
-                rs = statement.executeQuery(SELECT_ANONYMOUS_PROVENANCE_AUTHORITY);
-                if (!rs.next()) {
-                    statement.execute(INSERT_ANONYMOUS_PROVENANCE_AUTHORITY);
-                }
-
-                // commit any changes
-                connection.commit();
-            } catch (SQLException sqle) {
-                RepositoryUtils.rollback(connection, logger);
-                throw sqle;
-            } finally {
-                RepositoryUtils.closeQuietly(rs);
-                RepositoryUtils.closeQuietly(statement);
-                RepositoryUtils.closeQuietly(connection);
-            }
-        }
-
-        return connectionPool;
-    }
-
-    /**
-     * Get the database url for the specified database file.
-     *
-     * @param databaseFile
-     * @return
-     */
-    private String getDatabaseUrl(File databaseFile) {
-        String databaseUrl = "jdbc:h2:" + databaseFile + ";AUTOCOMMIT=OFF;DB_CLOSE_ON_EXIT=FALSE;LOCK_MODE=3";
-        String databaseUrlAppend = properties.getProperty(NiFiProperties.H2_URL_APPEND);
-        if (StringUtils.isNotBlank(databaseUrlAppend)) {
-            databaseUrl += databaseUrlAppend;
-        }
-        return databaseUrl;
-    }
-
-    @Override
-    public Class getObjectType() {
-        return JdbcConnectionPool.class;
-    }
-
-    @Override
-    public boolean isSingleton() {
-        return true;
-    }
-
-    public void setProperties(NiFiProperties properties) {
-        this.properties = properties;
-    }
-
-    /**
-     * Disposes resources.
-     */
-    public void shutdown() {
-
-        // shutdown the connection pool
-        if (connectionPool != null) {
-            try {
-                connectionPool.dispose();
-            } catch (Exception e) {
-                logger.warn("Unable to dispose of connection pool: " + e.getMessage());
-                if (logger.isDebugEnabled()) {
-                    logger.warn(StringUtils.EMPTY, e);
-                }
-            }
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java
deleted file mode 100644
index 5d6d222..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java
+++ /dev/null
@@ -1,74 +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.admin.dao;
-
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-import org.apache.nifi.action.Action;
-import org.apache.nifi.history.HistoryQuery;
-import org.apache.nifi.history.History;
-import org.apache.nifi.history.PreviousValue;
-
-/**
- * Action data access.
- */
-public interface ActionDAO {
-
-    /**
-     * Persists the specified action.
-     *
-     * @param action
-     * @throws DataAccessException
-     */
-    void createAction(Action action) throws DataAccessException;
-
-    /**
-     * Finds all actions that meet the specified criteria.
-     *
-     * @param actionQuery
-     * @return
-     * @throws DataAccessException
-     */
-    History findActions(HistoryQuery actionQuery) throws DataAccessException;
-
-    /**
-     * Finds the previous values for the specified property in the specified
-     * processor. Returns empty list if there are none.
-     *
-     * @param processorId
-     * @return
-     */
-    Map<String, List<PreviousValue>> getPreviousValues(String processorId);
-
-    /**
-     * Finds the specified action.
-     *
-     * @param actionId
-     * @return
-     * @throws DataAccessException
-     */
-    Action getAction(Integer actionId) throws DataAccessException;
-
-    /**
-     * Deletes all actions up to the specified end date.
-     *
-     * @param endDate
-     * @throws DataAccessException
-     */
-    void deleteActions(Date endDate) throws DataAccessException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/AuthorityDAO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/AuthorityDAO.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/AuthorityDAO.java
deleted file mode 100644
index 2992884..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/AuthorityDAO.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.admin.dao;
-
-import java.util.Set;
-import org.apache.nifi.authorization.Authority;
-
-/**
- * Authority data access.
- */
-public interface AuthorityDAO {
-
-    /**
-     * Finds all Authority for the specified user.
-     *
-     * @param userId
-     * @return
-     */
-    Set<Authority> findAuthoritiesByUserId(String userId) throws DataAccessException;
-
-    /**
-     * Creates a new Authorities for the specified user.
-     *
-     * @param authorities
-     * @param userId
-     */
-    void createAuthorities(Set<Authority> authorities, String userId) throws DataAccessException;
-
-    /**
-     * Removes all Authorities for the specified user.
-     *
-     * @param userId
-     * @throws DataAccessException
-     */
-    void deleteAuthorities(String userId) throws DataAccessException;
-
-    /**
-     * Removes the specified Authority.
-     *
-     * @param authorities
-     * @param userId
-     */
-    void deleteAuthorities(Set<Authority> authorities, String userId) throws DataAccessException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/DAOFactory.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/DAOFactory.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/DAOFactory.java
deleted file mode 100644
index dee4ef9..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/DAOFactory.java
+++ /dev/null
@@ -1,29 +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.admin.dao;
-
-/**
- *
- */
-public interface DAOFactory {
-
-    UserDAO getUserDAO();
-
-    ActionDAO getActionDAO();
-
-    AuthorityDAO getAuthorityDAO();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/DataAccessException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/DataAccessException.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/DataAccessException.java
deleted file mode 100644
index 05bf4af..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/DataAccessException.java
+++ /dev/null
@@ -1,39 +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.admin.dao;
-
-/**
- * Represents any error that might occur while administering NiFi accounts.
- */
-public class DataAccessException extends RuntimeException {
-
-    public DataAccessException(Throwable cause) {
-        super(cause);
-    }
-
-    public DataAccessException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public DataAccessException(String message) {
-        super(message);
-    }
-
-    public DataAccessException() {
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/UserDAO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/UserDAO.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/UserDAO.java
deleted file mode 100644
index 9ffab5d..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/UserDAO.java
+++ /dev/null
@@ -1,127 +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.admin.dao;
-
-import java.util.Date;
-import java.util.Set;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUser;
-
-/**
- * Defines the user data access object.
- */
-public interface UserDAO {
-
-    /**
-     * Determines whether there are any PENDING user accounts.
-     *
-     * @return
-     * @throws DataAccessException
-     */
-    Boolean hasPendingUserAccounts() throws DataAccessException;
-
-    /**
-     * Returns all users.
-     *
-     * @return
-     * @throws DataAccessException
-     */
-    Set<NiFiUser> findUsers() throws DataAccessException;
-
-    /**
-     * Returns all user groups.
-     *
-     * @return
-     * @throws DataAccessException
-     */
-    Set<String> findUserGroups() throws DataAccessException;
-
-    /**
-     * Returns all users for the specified group.
-     *
-     * @param group
-     * @return
-     * @throws DataAccessException
-     */
-    Set<NiFiUser> findUsersForGroup(String group) throws DataAccessException;
-
-    /**
-     * Returns the user with the specified id.
-     *
-     * @param id
-     * @return
-     * @throws DataAccessException
-     */
-    NiFiUser findUserById(String id) throws DataAccessException;
-
-    /**
-     * Returns the user with the specified DN.
-     *
-     * @param dn
-     * @return
-     */
-    NiFiUser findUserByDn(String dn) throws DataAccessException;
-
-    /**
-     * Creates a new user based off the specified NiFiUser.
-     *
-     * @param user
-     */
-    void createUser(NiFiUser user) throws DataAccessException;
-
-    /**
-     * Updates the specified NiFiUser.
-     *
-     * @param user
-     */
-    void updateUser(NiFiUser user) throws DataAccessException;
-
-    /**
-     * Deletes the specified user.
-     *
-     * @param id
-     * @throws DataAccessException
-     */
-    void deleteUser(String id) throws DataAccessException;
-
-    /**
-     * Sets the status of the specified group.
-     *
-     * @param group
-     * @param status
-     * @throws DataAccessException
-     */
-    void updateGroupStatus(String group, AccountStatus status) throws DataAccessException;
-
-    /**
-     * Sets the last verified time for all users in the specified group.
-     *
-     * @param group
-     * @param lastVerified
-     * @throws DataAccessException S
-     */
-    void updateGroupVerification(String group, Date lastVerified) throws DataAccessException;
-
-    /**
-     * Ungroups the specified group.
-     *
-     * @param group
-     * @throws DataAccessException
-     */
-    void ungroup(String group) throws DataAccessException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/DAOFactoryImpl.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/DAOFactoryImpl.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/DAOFactoryImpl.java
deleted file mode 100644
index 2f3de0e..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/DAOFactoryImpl.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.admin.dao.impl;
-
-import java.sql.Connection;
-import org.apache.nifi.admin.dao.ActionDAO;
-import org.apache.nifi.admin.dao.AuthorityDAO;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.UserDAO;
-
-/**
- *
- */
-public class DAOFactoryImpl implements DAOFactory {
-
-    private final Connection connection;
-
-    public DAOFactoryImpl(Connection connection) {
-        this.connection = connection;
-    }
-
-    @Override
-    public ActionDAO getActionDAO() {
-        return new StandardActionDAO(connection);
-    }
-
-    @Override
-    public AuthorityDAO getAuthorityDAO() {
-        return new StandardAuthorityDAO(connection);
-    }
-
-    @Override
-    public UserDAO getUserDAO() {
-        return new StandardUserDAO(connection);
-    }
-
-}


[17/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowResponseMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowResponseMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowResponseMessage.java
deleted file mode 100644
index 0d34dae..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowResponseMessage.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.cluster.protocol.message;
-
-import javax.xml.bind.annotation.XmlRootElement;
-
-import org.apache.nifi.cluster.protocol.StandardDataFlow;
-
-/**
- * @author unattributed
- */
-@XmlRootElement(name = "flowResponseMessage")
-public class FlowResponseMessage extends ProtocolMessage {
-    
-    private StandardDataFlow dataFlow;
-    
-    @Override
-    public MessageType getType() {
-        return MessageType.FLOW_RESPONSE;
-    }
-
-    public StandardDataFlow getDataFlow() {
-        return dataFlow;
-    }
-
-    public void setDataFlow(StandardDataFlow dataFlow) {
-        this.dataFlow = dataFlow;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/HeartbeatMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/HeartbeatMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/HeartbeatMessage.java
deleted file mode 100644
index 0064cb6..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/HeartbeatMessage.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.cluster.protocol.message;
-
-import org.apache.nifi.cluster.protocol.Heartbeat;
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- * @author unattributed
- */
-@XmlRootElement(name = "heartbeatMessage")
-public class HeartbeatMessage extends ProtocolMessage {
-    
-    private Heartbeat heartbeat;
-    
-    @Override
-    public MessageType getType() {
-        return MessageType.HEARTBEAT;
-    }
-
-    public Heartbeat getHeartbeat() {
-        return heartbeat;
-    }
-
-    public void setHeartbeat(Heartbeat heartbeat) {
-        this.heartbeat = heartbeat;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/MulticastProtocolMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/MulticastProtocolMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/MulticastProtocolMessage.java
deleted file mode 100644
index c6d2d44..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/MulticastProtocolMessage.java
+++ /dev/null
@@ -1,66 +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.cluster.protocol.message;
-
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- * Wraps a protocol message and an identifier for sending the message by way
- * multicast.  The identifier is necessary for the sender to identify a message
- * sent by it.
- * 
- * @author unattributed
- */
-@XmlRootElement(name = "multicastMessage")
-public class MulticastProtocolMessage extends ProtocolMessage {
-    
-    private ProtocolMessage protocolMessage;
-    
-    private String id;
-    
-    public MulticastProtocolMessage() {}
-
-    public MulticastProtocolMessage(final String id, final ProtocolMessage protocolMessage) {
-        this.protocolMessage = protocolMessage;
-        this.id = id;
-    }
-    
-    @Override
-    public MessageType getType() {
-        if(protocolMessage == null) {
-            return null;
-        }
-        return protocolMessage.getType();
-    }
-
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    public ProtocolMessage getProtocolMessage() {
-        return protocolMessage;
-    }
-
-    public void setProtocolMessage(ProtocolMessage protocolMessage) {
-        this.protocolMessage = protocolMessage;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/NodeBulletinsMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/NodeBulletinsMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/NodeBulletinsMessage.java
deleted file mode 100644
index 9237a92..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/NodeBulletinsMessage.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.cluster.protocol.message;
-
-import org.apache.nifi.cluster.protocol.NodeBulletins;
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- * @author unattributed
- */
-@XmlRootElement(name = "nodeBulletinsMessage")
-public class NodeBulletinsMessage extends ProtocolMessage {
-    
-    private NodeBulletins bulletins;
-    
-    @Override
-    public MessageType getType() {
-        return MessageType.BULLETINS;
-    }
-
-    public NodeBulletins getBulletins() {
-        return bulletins;
-    }
-
-    public void setBulletins(NodeBulletins bulletins) {
-        this.bulletins = bulletins;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PingMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PingMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PingMessage.java
deleted file mode 100644
index ee38deb..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PingMessage.java
+++ /dev/null
@@ -1,55 +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.cluster.protocol.message;
-
-import java.util.Date;
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- * @author unattributed
- */
-@XmlRootElement(name = "pingMessage")
-public class PingMessage extends ProtocolMessage {
-    
-    private String id;
-    
-    private Date date = new Date();
-
-    public PingMessage() {}
-    
-    public Date getDate() {
-        return date;
-    }
-
-    public void setDate(Date date) {
-        this.date = date;
-    }
-
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-    
-    @Override
-    public MessageType getType() {
-        return MessageType.PING;
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PrimaryRoleAssignmentMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PrimaryRoleAssignmentMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PrimaryRoleAssignmentMessage.java
deleted file mode 100644
index a289abc..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PrimaryRoleAssignmentMessage.java
+++ /dev/null
@@ -1,56 +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.cluster.protocol.message;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter;
-
-/**
- * @author unattributed
- */
-@XmlRootElement(name = "primaryRoleAssignmentMessage")
-public class PrimaryRoleAssignmentMessage extends ProtocolMessage {
-
-    private NodeIdentifier nodeId;
-
-    private boolean primary;
-    
-    @XmlJavaTypeAdapter(NodeIdentifierAdapter.class)
-    public NodeIdentifier getNodeId() {
-        return nodeId;
-    }
-
-    public void setNodeId(NodeIdentifier nodeId) {
-        this.nodeId = nodeId;
-    }
-
-    public boolean isPrimary() {
-        return primary;
-    }
-
-    public void setPrimary(boolean primary) {
-        this.primary = primary;
-    }
-   
-    @Override
-    public MessageType getType() {
-        return MessageType.PRIMARY_ROLE;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java
deleted file mode 100644
index 6bf2a13..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java
+++ /dev/null
@@ -1,61 +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.cluster.protocol.message;
-
-/**
- * @author unattributed
- */
-public abstract class ProtocolMessage {
-    private volatile String requestorDN;
-    
-    public static enum MessageType {
-        BULLETINS,
-        CONNECTION_REQUEST,
-        CONNECTION_RESPONSE,
-        CONTROLLER_STARTUP_FAILURE,
-        RECONNECTION_FAILURE,
-        DISCONNECTION_REQUEST,
-        EXCEPTION,
-        FLOW_REQUEST,
-        FLOW_RESPONSE,
-        HEARTBEAT,
-        PING,
-        PRIMARY_ROLE,
-        RECONNECTION_REQUEST,
-        RECONNECTION_RESPONSE,
-        SERVICE_BROADCAST,
-    }
-    
-    public abstract MessageType getType();
-    
-    /**
-     * Sets the DN of the entity making the request
-     * @param dn
-     */
-    public void setRequestorDN(final String dn) {
-        this.requestorDN = dn;
-    }
-    
-    /**
-     * Returns the DN of the entity that made the request, if using a secure socket. Otherwise, returns <code>null</code>
-     * @return
-     */
-    public String getRequestorDN() {
-        return requestorDN;
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionFailureMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionFailureMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionFailureMessage.java
deleted file mode 100644
index ba45e28..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionFailureMessage.java
+++ /dev/null
@@ -1,45 +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.cluster.protocol.message;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter;
-
-@XmlRootElement(name = "reconnectionFailureMessage")
-public class ReconnectionFailureMessage extends ExceptionMessage {
-    private NodeIdentifier nodeId;
-    
-    public ReconnectionFailureMessage() {}
-    
-    @Override
-    public MessageType getType() {
-        return MessageType.RECONNECTION_FAILURE;
-    }
-
-    @XmlJavaTypeAdapter(NodeIdentifierAdapter.class)
-    public NodeIdentifier getNodeId() {
-        return nodeId;
-    }
-
-    public void setNodeId(NodeIdentifier nodeId) {
-        this.nodeId = nodeId;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionRequestMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionRequestMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionRequestMessage.java
deleted file mode 100644
index eab3d5d..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionRequestMessage.java
+++ /dev/null
@@ -1,94 +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.cluster.protocol.message;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.StandardDataFlow;
-import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter;
-
-/**
- * @author unattributed
- */
-@XmlRootElement(name = "reconnectionRequestMessage")
-public class ReconnectionRequestMessage extends ProtocolMessage {
-
-    private NodeIdentifier nodeId;
-    private StandardDataFlow dataFlow;
-    private boolean primary;
-    private Integer managerRemoteSiteListeningPort;
-    private Boolean managerRemoteSiteCommsSecure;
-    private String instanceId;
-    
-    public ReconnectionRequestMessage() {}
-
-    @XmlJavaTypeAdapter(NodeIdentifierAdapter.class)
-    public NodeIdentifier getNodeId() {
-        return nodeId;
-    }
-
-    public void setNodeId(NodeIdentifier nodeId) {
-        this.nodeId = nodeId;
-    }
-
-    public StandardDataFlow getDataFlow() {
-        return dataFlow;
-    }
-
-    public void setDataFlow(StandardDataFlow dataFlow) {
-        this.dataFlow = dataFlow;
-    }
-
-    public boolean isPrimary() {
-        return primary;
-    }
-
-    public void setPrimary(boolean primary) {
-        this.primary = primary;
-    }
-    
-    @Override
-    public MessageType getType() {
-        return MessageType.RECONNECTION_REQUEST;
-    }
-    
-    public void setManagerRemoteSiteListeningPort(final Integer listeningPort) {
-        this.managerRemoteSiteListeningPort = listeningPort;
-    }
-    
-    public Integer getManagerRemoteSiteListeningPort() {
-        return managerRemoteSiteListeningPort;
-    }
-    
-    public void setManagerRemoteSiteCommsSecure(final Boolean remoteSiteCommsSecure) {
-        this.managerRemoteSiteCommsSecure = remoteSiteCommsSecure;
-    }
-    
-    public Boolean isManagerRemoteSiteCommsSecure() {
-        return managerRemoteSiteCommsSecure;
-    }
-    
-    public void setInstanceId(final String instanceId) {
-        this.instanceId = instanceId;
-    }
-    
-    public String getInstanceId() {
-        return instanceId;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionResponseMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionResponseMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionResponseMessage.java
deleted file mode 100644
index fd0f921..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionResponseMessage.java
+++ /dev/null
@@ -1,32 +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.cluster.protocol.message;
-
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- * This message is used an "ACK" for a ReconnectionRequestMessage
- */
-@XmlRootElement(name = "reconnectionResponseMessage")
-public class ReconnectionResponseMessage extends ProtocolMessage {
-
-    @Override
-    public MessageType getType() {
-        return MessageType.RECONNECTION_RESPONSE;
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ServiceBroadcastMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ServiceBroadcastMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ServiceBroadcastMessage.java
deleted file mode 100644
index 92708ba..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ServiceBroadcastMessage.java
+++ /dev/null
@@ -1,64 +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.cluster.protocol.message;
-
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- * @author unattributed
- */
-@XmlRootElement(name = "serviceBroadcastMessage")
-public class ServiceBroadcastMessage extends ProtocolMessage {
-
-    private String serviceName;
-    
-    private String address;
-    
-    private int port;
-    
-    public ServiceBroadcastMessage() {}
-
-    public String getServiceName() {
-        return serviceName;
-    }
-
-    public void setServiceName(String serviceName) {
-        this.serviceName = serviceName;
-    }
-
-    public String getAddress() {
-        return address;
-    }
-
-    public void setAddress(String address) {
-        this.address = address;
-    }
-
-    public int getPort() {
-        return port;
-    }
-
-    public void setPort(int port) {
-        this.port = port;
-    }
-
-    @Override
-    public MessageType getType() {
-        return MessageType.SERVICE_BROADCAST;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java
deleted file mode 100644
index fa201bb..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java
+++ /dev/null
@@ -1,60 +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.cluster.protocol.spring;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.NiFiProperties;
-import org.springframework.beans.factory.FactoryBean;
-
-/**
- * Factory bean for creating a singleton MulticastConfiguration instance.  
- */
-public class MulticastConfigurationFactoryBean implements FactoryBean {
-    
-    private MulticastConfiguration configuration;
-    private NiFiProperties properties;
-    
-    @Override
-    public Object getObject() throws Exception {
-        if(configuration == null) {
-            configuration = new MulticastConfiguration();
-            
-            final int timeout = (int) FormatUtils.getTimeDuration(properties.getClusterProtocolSocketTimeout(), TimeUnit.MILLISECONDS);
-            configuration.setSocketTimeout(timeout);
-            configuration.setReuseAddress(true);
-        }
-        return configuration;
-
-    }
-
-    @Override
-    public Class getObjectType() {
-        return MulticastConfiguration.class;
-    }
-
-    @Override
-    public boolean isSingleton() {
-        return true;
-    }
-    
-    public void setProperties(NiFiProperties properties) {
-        this.properties = properties;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java
deleted file mode 100644
index 5b5816d..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.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.cluster.protocol.spring;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.io.socket.SSLContextFactory;
-import org.apache.nifi.io.socket.ServerSocketConfiguration;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.NiFiProperties;
-
-import org.springframework.beans.factory.FactoryBean;
-
-/**
- * Factory bean for creating a singleton ServerSocketConfiguration instance.  
- */
-public class ServerSocketConfigurationFactoryBean implements FactoryBean<ServerSocketConfiguration> {
-    private ServerSocketConfiguration configuration;
-    private NiFiProperties properties;
-    
-    @Override
-    public ServerSocketConfiguration getObject() throws Exception {
-        if(configuration == null) {
-            configuration = new ServerSocketConfiguration();
-            configuration.setNeedClientAuth(properties.getNeedClientAuth());
-            
-            final int timeout = (int) FormatUtils.getTimeDuration(properties.getClusterProtocolSocketTimeout(), TimeUnit.MILLISECONDS);
-            configuration.setSocketTimeout(timeout);
-            configuration.setReuseAddress(true);
-            if(Boolean.valueOf(properties.getProperty(NiFiProperties.CLUSTER_PROTOCOL_IS_SECURE))) {
-                configuration.setSSLContextFactory(new SSLContextFactory(properties));
-            }
-        }
-        return configuration;
-
-    }
-
-    @Override
-    public Class<ServerSocketConfiguration> getObjectType() {
-        return ServerSocketConfiguration.class;
-    }
-
-    @Override
-    public boolean isSingleton() {
-        return true;
-    }
-    
-    public void setProperties(NiFiProperties properties) {
-        this.properties = properties;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/SocketConfigurationFactoryBean.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/SocketConfigurationFactoryBean.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/SocketConfigurationFactoryBean.java
deleted file mode 100644
index b438e44..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/SocketConfigurationFactoryBean.java
+++ /dev/null
@@ -1,66 +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.cluster.protocol.spring;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.io.socket.SSLContextFactory;
-import org.apache.nifi.io.socket.SocketConfiguration;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.NiFiProperties;
-
-import org.springframework.beans.factory.FactoryBean;
-
-/**
- * Factory bean for creating a singleton SocketConfiguration instance.  
- */
-public class SocketConfigurationFactoryBean implements FactoryBean<SocketConfiguration> {
-    
-    private SocketConfiguration configuration;
-    
-    private NiFiProperties properties;
-    
-    @Override
-    public SocketConfiguration getObject() throws Exception {
-        if(configuration == null) {
-            configuration = new SocketConfiguration();
-            
-            final int timeout = (int) FormatUtils.getTimeDuration(properties.getClusterProtocolSocketTimeout(), TimeUnit.MILLISECONDS);
-            configuration.setSocketTimeout(timeout);
-            configuration.setReuseAddress(true);
-            if(Boolean.valueOf(properties.getProperty(NiFiProperties.CLUSTER_PROTOCOL_IS_SECURE))) {
-                configuration.setSSLContextFactory(new SSLContextFactory(properties));
-            }
-        }
-        return configuration;
-
-    }
-
-    @Override
-    public Class<SocketConfiguration> getObjectType() {
-        return SocketConfiguration.class;
-    }
-
-    @Override
-    public boolean isSingleton() {
-        return true;
-    }
-    
-    public void setProperties(NiFiProperties properties) {
-        this.properties = properties;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml
deleted file mode 100644
index 07ea7a4..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml
+++ /dev/null
@@ -1,110 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<!-- marked as lazy so that cluster protocol beans are not created when applications runs in standalone mode -->
-<beans default-lazy-init="true"
-       xmlns="http://www.springframework.org/schema/beans"
-       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xmlns:util="http://www.springframework.org/schema/util"
-       xsi:schemaLocation="http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-3.1.xsd
-        http://www.springframework.org/schema/util http://www.springframework.org/schema/util/spring-util-3.1.xsd">
-
-    <!-- protocol context -->
-    <bean id="protocolContext" class="org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext">
-        <constructor-arg>
-            <util:constant static-field="org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils.JAXB_CONTEXT"/>
-        </constructor-arg>
-    </bean>
-    
-    <!-- socket configuration -->
-    <bean id="protocolSocketConfiguration" class="org.apache.nifi.cluster.protocol.spring.SocketConfigurationFactoryBean">
-        <property name="properties" ref="nifiProperties"/>
-    </bean>
-    
-    <!-- server socket configuration -->
-    <bean id="protocolServerSocketConfiguration" class="org.apache.nifi.cluster.protocol.spring.ServerSocketConfigurationFactoryBean">
-        <property name="properties" ref="nifiProperties"/>
-    </bean>
-    
-    <!-- multicast configuration -->
-    <bean id="protocolMulticastConfiguration" class="org.apache.nifi.cluster.protocol.spring.MulticastConfigurationFactoryBean">
-        <property name="properties" ref="nifiProperties"/>
-    </bean>
-
-    <!-- cluster manager protocol sender -->
-    <bean id="clusterManagerProtocolSender" class="org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderImpl">
-        <constructor-arg ref="protocolSocketConfiguration"/>
-        <constructor-arg ref="protocolContext"/>
-        <property name="handshakeTimeout">
-            <bean factory-bean="nifiProperties" factory-method="getClusterProtocolConnectionHandshakeTimeout"/>
-        </property>
-    </bean>
-    
-    <!-- cluster manager protocol listener -->
-    <bean id="clusterManagerProtocolListener" class="org.apache.nifi.cluster.protocol.impl.SocketProtocolListener">
-        <constructor-arg index="0">
-            <bean factory-bean="nifiProperties" factory-method="getClusterManagerProtocolThreads"/>
-        </constructor-arg>
-        <constructor-arg index="1">
-            <bean factory-bean="nifiProperties" factory-method="getClusterManagerProtocolPort"/>
-        </constructor-arg>
-        <constructor-arg ref="protocolServerSocketConfiguration" index="2"/>
-        <constructor-arg ref="protocolContext" index="3"/>
-    </bean>
-    
-    <!-- cluster manager sender/listener -->
-    <bean id="clusterManagerProtocolSenderListener" class="org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderListener">
-        <constructor-arg ref="clusterManagerProtocolSender"/>
-        <constructor-arg ref="clusterManagerProtocolListener"/>
-    </bean>
-    
-    <!-- node protocol sender -->
-    <bean id="nodeProtocolSender" class="org.apache.nifi.cluster.protocol.impl.NodeProtocolSenderImpl">
-        <constructor-arg ref="clusterManagerProtocolServiceLocator"/>
-        <constructor-arg ref="protocolSocketConfiguration"/>
-        <constructor-arg ref="protocolContext"/>
-    </bean>
-    
-    <!-- node protocol listener -->
-    <bean id="nodeProtocolListener" class="org.apache.nifi.cluster.protocol.impl.SocketProtocolListener">
-        <constructor-arg index="0">
-            <bean factory-bean="nifiProperties" factory-method="getClusterNodeProtocolThreads"/>
-        </constructor-arg>
-        <constructor-arg index="1">
-            <bean factory-bean="nifiProperties" factory-method="getClusterNodeProtocolPort"/>
-        </constructor-arg>
-        <constructor-arg ref="protocolServerSocketConfiguration" index="2"/>
-        <constructor-arg ref="protocolContext" index="3"/>
-    </bean>
-    
-    <!-- node sender/listener -->
-    <bean id="nodeProtocolSenderListener" class="org.apache.nifi.cluster.protocol.impl.NodeProtocolSenderListener">
-        <constructor-arg ref="nodeProtocolSender"/>
-        <constructor-arg ref="nodeProtocolListener"/>
-    </bean>
-    
-    <!-- cluster services broadcaster -->
-    <bean id="clusterServicesBroadcaster" class="org.apache.nifi.cluster.protocol.impl.ClusterServicesBroadcaster">
-        <constructor-arg index="0">
-            <bean factory-bean="nifiProperties" factory-method="getClusterProtocolMulticastAddress"/>
-        </constructor-arg>
-        <constructor-arg ref="protocolMulticastConfiguration" index="1"/>
-        <constructor-arg ref="protocolContext" index="2"/>
-        <constructor-arg index="3">
-            <bean factory-bean="nifiProperties" factory-method="getClusterProtocolMulticastServiceBroadcastDelay"/>
-        </constructor-arg>
-    </bean>
- 
-</beans>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java
deleted file mode 100644
index 59837c1..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java
+++ /dev/null
@@ -1,134 +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.cluster.protocol.impl;
-
-import org.apache.nifi.cluster.protocol.impl.SocketProtocolListener;
-import org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderImpl;
-import java.io.IOException;
-import java.net.InetAddress;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
-import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
-import org.apache.nifi.cluster.protocol.message.PingMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.io.socket.ServerSocketConfiguration;
-import org.apache.nifi.io.socket.SocketConfiguration;
-import org.junit.After;
-import static org.junit.Assert.*;
-import org.junit.Before;
-import org.junit.Test;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.*;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * @author unattributed
- */
-public class ClusterManagerProtocolSenderImplTest {
-    
-    private InetAddress address;
-    
-    private int port;
-    
-    private SocketProtocolListener listener;
-    
-    private ClusterManagerProtocolSenderImpl sender;
-    
-    private ProtocolHandler mockHandler;
-    
-    @Before
-    public void setup() throws IOException {
-        
-        address = InetAddress.getLocalHost();
-        ServerSocketConfiguration serverSocketConfiguration = new ServerSocketConfiguration();
-        serverSocketConfiguration.setSocketTimeout(2000);
-
-        mockHandler = mock(ProtocolHandler.class);
-        
-        ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
-        
-        listener = new SocketProtocolListener(5, 0, serverSocketConfiguration, protocolContext);
-        listener.addHandler(mockHandler);
-        listener.start();
-        
-        port = listener.getPort();
-        
-        SocketConfiguration socketConfiguration = new SocketConfiguration();
-        sender = new ClusterManagerProtocolSenderImpl(socketConfiguration, protocolContext);
-    }
-    
-    @After
-    public void teardown() throws IOException {
-        if(listener.isRunning()) {
-            listener.stop();
-        }
-    }
-    
-    @Test
-    public void testRequestFlow() throws Exception {
-        
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new FlowResponseMessage());
-        FlowRequestMessage request = new FlowRequestMessage();
-        request.setNodeId(new NodeIdentifier("id", "api-address", 1, address.getHostAddress(), port));
-        FlowResponseMessage response = sender.requestFlow(request);
-        assertNotNull(response);
-    }
-    
-    @Test
-    public void testRequestFlowWithBadResponseMessage() throws Exception {
-        
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new PingMessage());
-        FlowRequestMessage request = new FlowRequestMessage();
-        request.setNodeId(new NodeIdentifier("id", "api-address", 1, address.getHostAddress(), port));
-        try {
-            sender.requestFlow(request);
-            fail("failed to throw exception");
-        } catch(ProtocolException pe) {}
-        
-    }
-    
-    @Test
-    public void testRequestFlowDelayedResponse() throws Exception {
-        
-        final int time = 250;
-        sender.getSocketConfiguration().setSocketTimeout(time);
-        
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenAnswer(new Answer<FlowResponseMessage>() {
-            @Override
-            public FlowResponseMessage answer(InvocationOnMock invocation) throws Throwable {
-                Thread.sleep(time * 3);
-                return new FlowResponseMessage();
-            }
-        });
-        FlowRequestMessage request = new FlowRequestMessage();
-        request.setNodeId(new NodeIdentifier("id", "api-address", 1, address.getHostAddress(), port));
-        try {
-            sender.requestFlow(request);
-            fail("failed to throw exception");
-        } catch(ProtocolException pe) {}
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java
deleted file mode 100644
index e3703e2..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java
+++ /dev/null
@@ -1,135 +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.cluster.protocol.impl;
-
-import org.apache.nifi.cluster.protocol.impl.ClusterServiceDiscovery;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.net.DatagramPacket;
-import java.net.InetSocketAddress;
-import java.net.MulticastSocket;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.PingMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.apache.nifi.io.socket.multicast.MulticastUtils;
-import org.junit.After;
-import static org.junit.Assert.*;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-/**
- * @author unattributed
- */
-public class ClusterServiceDiscoveryTest {
-    
-    private ClusterServiceDiscovery discovery;
-    
-    private String serviceName;
-    
-    private MulticastSocket socket;
-    
-    private InetSocketAddress multicastAddress;
-    
-    private MulticastConfiguration configuration;
-    
-    private ProtocolContext protocolContext;
-    
-    @Before
-    public void setup() throws Exception {
-
-        serviceName = "some-service";
-        multicastAddress = new InetSocketAddress("225.1.1.1", 22222);
-        configuration = new MulticastConfiguration();
-        
-        protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
-        
-        discovery = new ClusterServiceDiscovery(serviceName, multicastAddress, configuration, protocolContext);
-        discovery.start();
-
-        socket = MulticastUtils.createMulticastSocket(multicastAddress.getPort(), configuration);
-    }
-    
-    @After
-    public void teardown() throws IOException {
-        try {
-            if(discovery.isRunning()) {
-                discovery.stop();
-            }
-        } finally {
-            MulticastUtils.closeQuietly(socket);
-        }
-    }
-    
-    @Ignore("Test needs to be fixed.  Requires an active network connection")
-    @Test
-    public void testGetAddressOnStartup() {
-        assertNull(discovery.getService());
-    }   
-            
-    @Ignore("This test has an NPE after ignoring another...perhaps has a bad inter-test dependency")
-    @Test
-    public void testGetAddressAfterBroadcast() throws Exception {
-        
-        ServiceBroadcastMessage msg = new ServiceBroadcastMessage();
-        msg.setServiceName("some-service");
-        msg.setAddress("3.3.3.3");
-        msg.setPort(1234);
-        
-        // marshal message to output stream
-        ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller();
-        ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        marshaller.marshal(msg, baos);
-        byte[] requestPacketBytes = baos.toByteArray();
-        DatagramPacket packet = new DatagramPacket(requestPacketBytes, requestPacketBytes.length, multicastAddress);
-        socket.send(packet);
-        
-        Thread.sleep(250);
-       
-        InetSocketAddress updatedAddress = discovery.getService().getServiceAddress();
-        assertEquals("some-service", discovery.getServiceName());
-        assertEquals("3.3.3.3", updatedAddress.getHostName());
-        assertEquals(1234, updatedAddress.getPort());
-        
-    }
-    
-    @Ignore("Test needs to be fixed.  Requires an active network connection")
-    @Test
-    public void testBadBroadcastMessage() throws Exception {
-        
-        ProtocolMessage msg = new PingMessage();
-        
-        // marshal message to output stream
-        ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller();
-        ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        marshaller.marshal(msg, baos);
-        byte[] requestPacketBytes = baos.toByteArray();
-        DatagramPacket packet = new DatagramPacket(requestPacketBytes, requestPacketBytes.length, multicastAddress);
-        socket.send(packet);
-        
-        Thread.sleep(250);
-       
-        assertNull(discovery.getService());
-        
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java
deleted file mode 100644
index b1c156b..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java
+++ /dev/null
@@ -1,121 +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.cluster.protocol.impl;
-
-import org.apache.nifi.cluster.protocol.impl.ClusterServiceLocator;
-import org.apache.nifi.cluster.protocol.impl.ClusterServiceDiscovery;
-import java.net.InetSocketAddress;
-import java.util.concurrent.TimeUnit;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-import static org.junit.Assert.*;
-import org.junit.Before;
-import org.junit.Test;
-import static org.mockito.Mockito.*;
-import org.mockito.stubbing.OngoingStubbing;
-
-public class ClusterServiceLocatorTest {
-    
-    private ClusterServiceDiscovery mockServiceDiscovery;
-    
-    private int fixedPort;
-    
-    private DiscoverableService fixedService;
-    
-    private ClusterServiceLocator serviceDiscoveryLocator;
-    
-    private ClusterServiceLocator serviceDiscoveryFixedPortLocator;
-    
-    private ClusterServiceLocator fixedServiceLocator;
-    
-    @Before
-    public void setup() throws Exception {
-        
-        fixedPort = 1;
-        mockServiceDiscovery = mock(ClusterServiceDiscovery.class);
-        fixedService = new DiscoverableServiceImpl("some-service", InetSocketAddress.createUnresolved("some-host", 20));
-        
-        serviceDiscoveryLocator = new ClusterServiceLocator(mockServiceDiscovery);
-        serviceDiscoveryFixedPortLocator = new ClusterServiceLocator(mockServiceDiscovery, fixedPort);
-        fixedServiceLocator = new ClusterServiceLocator(fixedService);
-        
-    }
-    
-    @Test
-    public void getServiceWhenServiceDiscoveryNotStarted() {
-        assertNull(serviceDiscoveryLocator.getService());
-    }
-    
-    @Test
-    public void getServiceWhenServiceDiscoveryFixedPortNotStarted() {
-        assertNull(serviceDiscoveryLocator.getService());
-    }
-    
-    @Test
-    public void getServiceWhenFixedServiceNotStarted() {
-        assertEquals(fixedService, fixedServiceLocator.getService());
-    }
-    
-    @Test
-    public void getServiceNotOnFirstAttempt() {
-                
-        ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig();
-        config.setNumAttempts(2);
-        config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS);
-        config.setTimeBetweenAttempts(1);
-        
-        serviceDiscoveryLocator.setAttemptsConfig(config);
-        
-        OngoingStubbing<DiscoverableService> stubbing = null;
-        for(int i = 0; i < config.getNumAttempts() - 1; i++) {
-            if(stubbing == null) {
-                stubbing = when(mockServiceDiscovery.getService()).thenReturn(null);
-            } else {
-                stubbing.thenReturn(null);
-            }
-        }
-        stubbing.thenReturn(fixedService);
-        
-        assertEquals(fixedService, serviceDiscoveryLocator.getService());
-        
-    }
-    
-    @Test
-    public void getServiceNotOnFirstAttemptWithFixedPort() {
-        
-        ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig();
-        config.setNumAttempts(2);
-        config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS);
-        config.setTimeBetweenAttempts(1);
-        
-        serviceDiscoveryFixedPortLocator.setAttemptsConfig(config);
-        
-        OngoingStubbing<DiscoverableService> stubbing = null;
-        for(int i = 0; i < config.getNumAttempts() - 1; i++) {
-            if(stubbing == null) {
-                stubbing = when(mockServiceDiscovery.getService()).thenReturn(null);
-            } else {
-                stubbing.thenReturn(null);
-            }
-        }
-        stubbing.thenReturn(fixedService);
-        
-        InetSocketAddress resultAddress = InetSocketAddress.createUnresolved(fixedService.getServiceAddress().getHostName(), fixedPort);
-        DiscoverableService resultService = new DiscoverableServiceImpl(fixedService.getServiceName(), resultAddress);
-        assertEquals(resultService, serviceDiscoveryFixedPortLocator.getService());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java
deleted file mode 100644
index ec1f26d..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java
+++ /dev/null
@@ -1,133 +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.cluster.protocol.impl;
-
-import org.apache.nifi.cluster.protocol.impl.ClusterServicesBroadcaster;
-import org.apache.nifi.cluster.protocol.impl.MulticastProtocolListener;
-import java.net.InetSocketAddress;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.junit.After;
-import static org.junit.Assert.*;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-/**
- * @author unattributed
- */
-public class ClusterServicesBroadcasterTest {
-    
-    private ClusterServicesBroadcaster broadcaster;
-    
-    private MulticastProtocolListener listener;
-    
-    private DummyProtocolHandler handler;
-    
-    private InetSocketAddress multicastAddress;
-    
-    private DiscoverableService broadcastedService;
-
-    private ProtocolContext protocolContext;
-    
-    private MulticastConfiguration configuration;
-    
-    @Before
-    public void setup() throws Exception {
-
-        broadcastedService = new DiscoverableServiceImpl("some-service", new InetSocketAddress("localhost", 11111));
-        
-        multicastAddress = new InetSocketAddress("225.1.1.1", 22222);
-        
-        configuration = new MulticastConfiguration();
-        
-        protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
-        
-        broadcaster = new ClusterServicesBroadcaster(multicastAddress, configuration, protocolContext, "500 ms");
-        broadcaster.addService(broadcastedService);
-        
-        handler = new DummyProtocolHandler();
-        listener = new MulticastProtocolListener(5, multicastAddress, configuration, protocolContext);
-        listener.addHandler(handler);
-    }
-    
-    @After
-    public void teardown() {
-        
-        if(broadcaster.isRunning()) {
-            broadcaster.stop();
-        }
-        
-        try {
-            if(listener.isRunning()) {
-                listener.stop();
-            }
-        } catch(Exception ex) {
-            ex.printStackTrace(System.out);
-        }
-        
-    }
-    
-    @Ignore("fails needs to be fixed")
-    @Test
-    public void testBroadcastReceived() throws Exception {
-        
-        broadcaster.start();
-        listener.start();
-        
-        Thread.sleep(1000);
-        
-        listener.stop();
-        
-        assertNotNull(handler.getProtocolMessage());
-        assertEquals(ProtocolMessage.MessageType.SERVICE_BROADCAST, handler.getProtocolMessage().getType());
-        final ServiceBroadcastMessage msg = (ServiceBroadcastMessage) handler.getProtocolMessage();
-        assertEquals(broadcastedService.getServiceName(), msg.getServiceName());
-        assertEquals(broadcastedService.getServiceAddress().getHostName(), msg.getAddress());
-        assertEquals(broadcastedService.getServiceAddress().getPort(), msg.getPort());
-    }
-    
-    private class DummyProtocolHandler implements ProtocolHandler {
-
-        private ProtocolMessage protocolMessage;
-        
-        @Override
-        public boolean canHandle(ProtocolMessage msg) {
-            return true;
-        }
-
-        @Override
-        public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
-            this.protocolMessage = msg;
-            return null;
-        }
-        
-        public ProtocolMessage getProtocolMessage() {
-            return protocolMessage;
-        }
-        
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java
deleted file mode 100644
index 4233d88..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java
+++ /dev/null
@@ -1,171 +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.cluster.protocol.impl;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.net.DatagramPacket;
-import java.net.InetSocketAddress;
-import java.net.MulticastSocket;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.MulticastProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.PingMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.apache.nifi.io.socket.multicast.MulticastUtils;
-import org.junit.After;
-import static org.junit.Assert.*;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-/**
- * @author unattributed
- */
-public class MulticastProtocolListenerTest {
-    
-    private MulticastProtocolListener listener;
-    
-    private MulticastSocket socket;
-    
-    private InetSocketAddress address;
-    
-    private MulticastConfiguration configuration;
-    
-    private ProtocolContext protocolContext;
-    
-    @Before
-    public void setup() throws Exception {
-
-        address = new InetSocketAddress("226.1.1.1", 60000);
-        configuration = new MulticastConfiguration();
-        
-        protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
-        
-        listener = new MulticastProtocolListener(5, address, configuration, protocolContext);
-        listener.start();
-
-        socket = MulticastUtils.createMulticastSocket(address.getPort(), configuration);
-    }
-    
-    @After
-    public void teardown() throws IOException {
-        try {
-            if(listener.isRunning()) {
-                listener.stop();
-            }
-        } finally {
-            MulticastUtils.closeQuietly(socket);
-        }
-    }
-    
-    @Ignore("Test needs to be reworked.  Fails if on a system without actiev network connection")
-    @Test
-    public void testBadRequest() throws Exception {
-        DelayedProtocolHandler handler = new DelayedProtocolHandler(0);
-        listener.addHandler(handler);
-        DatagramPacket packet = new DatagramPacket(new byte[] {5}, 1, address);
-        socket.send(packet);
-        Thread.sleep(250);
-        assertEquals(0, handler.getMessages().size());
-    }
-    
-    @Ignore("this test works sometimes and fails others - needs work to be reliable")
-    @Test
-    public void testRequest() throws Exception {
-
-        ReflexiveProtocolHandler handler = new ReflexiveProtocolHandler();
-        listener.addHandler(handler);
-        
-        ProtocolMessage msg = new PingMessage();
-        MulticastProtocolMessage multicastMsg = new MulticastProtocolMessage("some-id", msg);
-
-        // marshal message to output stream
-        ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller();
-        ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        marshaller.marshal(multicastMsg, baos);
-        byte[] requestPacketBytes = baos.toByteArray();
-        DatagramPacket packet = new DatagramPacket(requestPacketBytes, requestPacketBytes.length, address);
-        socket.send(packet);
-
-        Thread.sleep(250);
-        assertEquals(1, handler.getMessages().size());
-        assertEquals(msg.getType(), handler.getMessages().get(0).getType());
-        
-    }
-    
-    private class ReflexiveProtocolHandler implements ProtocolHandler {
-        
-        private List<ProtocolMessage> messages = new ArrayList<>();
-        
-        @Override
-        public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
-            messages.add(msg);
-            return msg;
-        }
-
-        @Override
-        public boolean canHandle(ProtocolMessage msg) {
-            return true;
-        }
-        
-        public List<ProtocolMessage> getMessages() {
-            return messages;
-        }
-        
-    }
-
-    private class DelayedProtocolHandler implements ProtocolHandler {
-            
-        private int delay = 0;
-        
-        private List<ProtocolMessage> messages = new ArrayList<>();
-            
-        public DelayedProtocolHandler(int delay) {
-            this.delay = delay;
-        }
-        
-        @Override
-        public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
-            try {
-                messages.add(msg);
-                Thread.sleep(delay);
-                return null;
-            } catch(final InterruptedException ie) {
-                throw new ProtocolException(ie);
-            }
-
-        }
-
-        @Override
-        public boolean canHandle(ProtocolMessage msg) {
-            return true;
-        }
-        
-        public List<ProtocolMessage> getMessages() {
-            return messages;
-        }
-        
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java
deleted file mode 100644
index 1c5ba9e..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java
+++ /dev/null
@@ -1,203 +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.cluster.protocol.impl;
-
-import org.apache.nifi.cluster.protocol.impl.ClusterServiceLocator;
-import org.apache.nifi.cluster.protocol.impl.SocketProtocolListener;
-import org.apache.nifi.cluster.protocol.impl.NodeProtocolSenderImpl;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.UUID;
-
-import org.apache.nifi.cluster.protocol.ConnectionRequest;
-import org.apache.nifi.cluster.protocol.ConnectionResponse;
-import org.apache.nifi.cluster.protocol.Heartbeat;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.StandardDataFlow;
-import org.apache.nifi.cluster.protocol.UnknownServiceAddressException;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage;
-import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage;
-import org.apache.nifi.cluster.protocol.message.ControllerStartupFailureMessage;
-import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
-import org.apache.nifi.cluster.protocol.message.PingMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.io.socket.ServerSocketConfiguration;
-import org.apache.nifi.io.socket.SocketConfiguration;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * @author unattributed
- */
-@Ignore("Randomly tests... probably timing-specific")
-public class NodeProtocolSenderImplTest {
-    
-    private SocketProtocolListener listener;
-    
-    private NodeProtocolSenderImpl sender;
-    
-    private DiscoverableService service;
-    
-    private ServerSocketConfiguration serverSocketConfiguration;
-    
-    private ClusterServiceLocator mockServiceLocator;
-    
-    private ProtocolHandler mockHandler;
-    
-    private NodeIdentifier nodeIdentifier;
-    
-    @Before
-    public void setup() throws IOException {
-        
-        serverSocketConfiguration = new ServerSocketConfiguration();
-
-        mockServiceLocator = mock(ClusterServiceLocator.class);
-        mockHandler = mock(ProtocolHandler.class);
-        
-        nodeIdentifier = new NodeIdentifier("1", "localhost", 1234, "localhost", 5678);
-        
-        ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
-        
-        listener = new SocketProtocolListener(5, 0, serverSocketConfiguration, protocolContext);
-        listener.setShutdownListenerSeconds(3);
-        listener.addHandler(mockHandler);
-        listener.start();
-        
-        service = new DiscoverableServiceImpl("some-service", new InetSocketAddress("localhost", listener.getPort()));
-        
-        SocketConfiguration socketConfiguration = new SocketConfiguration();
-        socketConfiguration.setReuseAddress(true);
-        sender = new NodeProtocolSenderImpl(mockServiceLocator, socketConfiguration, protocolContext);
-    }
-    
-    @After
-    public void teardown() throws IOException {
-        if(listener.isRunning()) {
-            listener.stop();
-        }
-    }
-    
-    @Test
-    public void testConnect() throws Exception {
-        
-        when(mockServiceLocator.getService()).thenReturn(service);
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        ConnectionResponseMessage mockMessage = new ConnectionResponseMessage();
-        mockMessage.setConnectionResponse(new ConnectionResponse(nodeIdentifier, new StandardDataFlow("flow".getBytes("UTF-8"), new byte[0], new byte[0]), false, null, null, UUID.randomUUID().toString()));
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(mockMessage);
-        
-        ConnectionRequestMessage request = new ConnectionRequestMessage();
-        request.setConnectionRequest(new ConnectionRequest(nodeIdentifier));
-        ConnectionResponseMessage response = sender.requestConnection(request);
-        assertNotNull(response);
-    }
-    
-    @Test(expected = UnknownServiceAddressException.class)
-    public void testConnectNoClusterManagerAddress() throws Exception {
-        
-        when(mockServiceLocator.getService()).thenReturn(null);
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new ConnectionResponseMessage());
-        
-        ConnectionRequestMessage request = new ConnectionRequestMessage();
-        request.setConnectionRequest(new ConnectionRequest(nodeIdentifier));
-
-        sender.requestConnection(request);
-        fail("failed to throw exception");
-    }
-    
-    @Test(expected = ProtocolException.class)
-    public void testConnectBadResponse() throws Exception {
-        
-        when(mockServiceLocator.getService()).thenReturn(service);
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new PingMessage());
-        
-        ConnectionRequestMessage request = new ConnectionRequestMessage();
-        request.setConnectionRequest(new ConnectionRequest(nodeIdentifier));
-        
-        sender.requestConnection(request);
-        fail("failed to throw exception");
-        
-    }
-    
-    @Test(expected = ProtocolException.class)
-    public void testConnectDelayedResponse() throws Exception {
-        
-        final int time = 250;
-        sender.getSocketConfiguration().setSocketTimeout(time);
-        when(mockServiceLocator.getService()).thenReturn(service);
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenAnswer(new Answer<ConnectionResponseMessage>() {
-            @Override
-            public ConnectionResponseMessage answer(InvocationOnMock invocation) throws Throwable {
-                Thread.sleep(time * 3);
-                return new ConnectionResponseMessage();
-            }
-        });
-        ConnectionRequestMessage request = new ConnectionRequestMessage();
-        request.setConnectionRequest(new ConnectionRequest(nodeIdentifier));
-
-        sender.requestConnection(request);
-        fail("failed to throw exception");
-        
-    }
-    
-    @Test
-    public void testHeartbeat() throws Exception {
-        
-        when(mockServiceLocator.getService()).thenReturn(service);
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(null);
-        
-        HeartbeatMessage hb = new HeartbeatMessage();
-        hb.setHeartbeat(new Heartbeat(new NodeIdentifier("id", "localhost", 3, "localhost", 4), false, false, new byte[] {1, 2, 3}));
-        sender.heartbeat(hb);
-    }
-    
-    @Test
-    public void testNotifyControllerStartupFailure() throws Exception {
-        
-        when(mockServiceLocator.getService()).thenReturn(service);
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(null);
-        
-        ControllerStartupFailureMessage msg = new ControllerStartupFailureMessage();
-        msg.setNodeId(new NodeIdentifier("some-id", "some-addr", 1, "some-addr", 1));
-        msg.setExceptionMessage("some exception");
-        sender.notifyControllerStartupFailure(msg);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/DelayedProtocolHandler.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/DelayedProtocolHandler.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/DelayedProtocolHandler.java
deleted file mode 100644
index 07ee83a..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/DelayedProtocolHandler.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.cluster.protocol.impl.testutils;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-
-/**
- * @author unattributed
- */
-public class DelayedProtocolHandler implements ProtocolHandler {
-
-    private int delay = 0;
-    private List<ProtocolMessage> messages = new ArrayList<>();
-
-    public DelayedProtocolHandler(int delay) {
-        this.delay = delay;
-    }
-
-    @Override
-    public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
-        try {
-            messages.add(msg);
-            Thread.sleep(delay);
-            return null;
-        } catch (final InterruptedException ie) {
-            throw new ProtocolException(ie);
-        }
-
-    }
-
-    @Override
-    public boolean canHandle(ProtocolMessage msg) {
-        return true;
-    }
-
-    public List<ProtocolMessage> getMessages() {
-        return messages;
-    }
-}


[11/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpRequest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpRequest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpRequest.java
deleted file mode 100644
index 35380dd..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpRequest.java
+++ /dev/null
@@ -1,239 +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.cluster.manager.testutils;
-
-import java.io.IOException;
-import java.io.Reader;
-import java.io.UnsupportedEncodingException;
-import java.net.URLDecoder;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import javax.ws.rs.HttpMethod;
-import javax.ws.rs.core.MediaType;
-import org.apache.commons.lang3.StringUtils;
-
-/**
- * Encapsulates an HTTP request. The toString method returns the
- * specification-compliant request.
- *
- * @author unattributed
- */
-public class HttpRequest {
-
-    private String method;
-    private String uri;
-    private String rawUri;
-    private String version;
-    private String body;
-    private String rawRequest;
-    private Map<String, String> headers = new HashMap<>();
-    private Map<String, List<String>> parameters = new HashMap<>();
-
-    public static HttpRequestBuilder createFromRequestLine(final String requestLine) {
-        return new HttpRequestBuilder(requestLine);
-    }
-
-    public String getBody() {
-        return body;
-    }
-
-    public Map<String, String> getHeaders() {
-        return Collections.unmodifiableMap(headers);
-    }
-
-    public String getHeaderValue(final String header) {
-        for (final Map.Entry<String, String> entry : getHeaders().entrySet()) {
-            if (entry.getKey().equalsIgnoreCase(header)) {
-                return entry.getValue();
-            }
-        }
-        return null;
-    }
-
-    public String getMethod() {
-        return method;
-    }
-
-    public Map<String, List<String>> getParameters() {
-        final Map<String, List<String>> result = new HashMap<>();
-        for (final Map.Entry<String, List<String>> entry : parameters.entrySet()) {
-            result.put(entry.getKey(), Collections.unmodifiableList(entry.getValue()));
-        }
-        return Collections.unmodifiableMap(result);
-    }
-
-    public String getUri() {
-        return uri;
-    }
-
-    public String getRawUri() {
-        return rawUri;
-    }
-
-    public String getVersion() {
-        return version;
-    }
-
-    @Override
-    public String toString() {
-        return rawRequest;
-    }
-
-    /**
-     * A builder for constructing basic HTTP requests. It handles only enough of
-     * the HTTP specification to support basic unit testing, and it should not
-     * be used otherwise.
-     */
-    public static class HttpRequestBuilder {
-
-        private String method;
-        private String uri;
-        private String rawUri;
-        private String version;
-        private Map<String, String> headers = new HashMap<>();
-        private Map<String, List<String>> parameters = new HashMap<>();
-        private int contentLength = 0;
-        private String contentType;
-        private String body = "";
-        private StringBuilder rawRequest = new StringBuilder();
-
-        private HttpRequestBuilder(final String requestLine) {
-
-            final String[] tokens = requestLine.split(" ");
-            if (tokens.length != 3) {
-                throw new IllegalArgumentException("Invalid HTTP Request Line: " + requestLine);
-            }
-
-            method = tokens[0];
-            if (HttpMethod.GET.equalsIgnoreCase(method) || HttpMethod.HEAD.equalsIgnoreCase(method) || HttpMethod.DELETE.equalsIgnoreCase(method) || HttpMethod.OPTIONS.equalsIgnoreCase(method)) {
-                final int queryIndex = tokens[1].indexOf("?");
-                if (queryIndex > -1) {
-                    uri = tokens[1].substring(0, queryIndex);
-                    addParameters(tokens[1].substring(queryIndex + 1));
-                } else {
-                    uri = tokens[1];
-                }
-            }
-            rawUri = tokens[1];
-            version = tokens[2];
-            rawRequest.append(requestLine).append("\n");
-        }
-
-        private void addHeader(final String key, final String value) {
-            if (key.contains(" ")) {
-                throw new IllegalArgumentException("Header key may not contain spaces.");
-            } else if ("content-length".equalsIgnoreCase(key)) {
-                contentLength = (StringUtils.isBlank(value.trim())) ? 0 : Integer.parseInt(value.trim());
-            } else if ("content-type".equalsIgnoreCase(key)) {
-                contentType = value.trim();
-            }
-            headers.put(key, value);
-        }
-
-        public void addHeader(final String header) {
-            final int firstColonIndex = header.indexOf(":");
-            if (firstColonIndex < 0) {
-                throw new IllegalArgumentException("Invalid HTTP Header line: " + header);
-            }
-            addHeader(header.substring(0, firstColonIndex), header.substring(firstColonIndex + 1));
-            rawRequest.append(header).append("\n");
-        }
-
-        // final because constructor calls it
-        public final void addParameters(final String queryString) {
-
-            if (StringUtils.isBlank(queryString)) {
-                return;
-            }
-
-            final String normQueryString;
-            if (queryString.startsWith("?")) {
-                normQueryString = queryString.substring(1);
-            } else {
-                normQueryString = queryString;
-            }
-            final String[] keyValuePairs = normQueryString.split("&");
-            for (final String keyValuePair : keyValuePairs) {
-                final String[] keyValueTokens = keyValuePair.split("=");
-                try {
-                    addParameter(
-                            URLDecoder.decode(keyValueTokens[0], "utf-8"),
-                            URLDecoder.decode(keyValueTokens[1], "utf-8")
-                    );
-                } catch (UnsupportedEncodingException use) {
-                    throw new RuntimeException(use);
-                }
-            }
-        }
-
-        public void addParameter(final String key, final String value) {
-
-            if (key.contains(" ")) {
-                throw new IllegalArgumentException("Parameter key may not contain spaces: " + key);
-            }
-
-            final List<String> values;
-            if (parameters.containsKey(key)) {
-                values = parameters.get(key);
-            } else {
-                values = new ArrayList<>();
-                parameters.put(key, values);
-            }
-            values.add(value);
-        }
-
-        public void addBody(final Reader reader) throws IOException {
-
-            if (contentLength <= 0) {
-                return;
-            }
-
-            final char[] buf = new char[contentLength];
-            int offset = 0;
-            int numRead = 0;
-            while (offset < buf.length && (numRead = reader.read(buf, offset, buf.length - offset)) >= 0) {
-                offset += numRead;
-            }
-            body = new String(buf);
-            rawRequest.append("\n");
-            rawRequest.append(body);
-        }
-
-        public HttpRequest build() throws UnsupportedEncodingException {
-
-            if (HttpMethod.GET.equalsIgnoreCase(method) == false && HttpMethod.HEAD.equalsIgnoreCase(method) == false && contentType.equalsIgnoreCase(MediaType.APPLICATION_FORM_URLENCODED)) {
-                addParameters(body);
-            }
-
-            final HttpRequest request = new HttpRequest();
-            request.method = this.method;
-            request.uri = this.uri;
-            request.rawUri = this.rawUri;
-            request.version = this.version;
-            request.headers.putAll(this.headers);
-            request.parameters.putAll(this.parameters);
-            request.body = this.body;
-            request.rawRequest = this.rawRequest.toString();
-
-            return request;
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponse.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponse.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponse.java
deleted file mode 100644
index 3aa2931..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponse.java
+++ /dev/null
@@ -1,93 +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.cluster.manager.testutils;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import javax.ws.rs.core.Response.Status;
-
-/**
- * Encapsulates an HTTP response. The toString method returns the
- * specification-compliant response.
- *
- * @author unattributed
- */
-public class HttpResponse {
-
-    private final Status status;
-    private final String entity;
-    private final Map<String, String> headers = new HashMap<>();
-
-    public HttpResponse(final Status status, final String entity) {
-        this.status = status;
-        this.entity = entity;
-        headers.put("content-length", String.valueOf(entity.getBytes().length));
-    }
-
-    public String getEntity() {
-        return entity;
-    }
-
-    public Status getStatus() {
-        return status;
-    }
-
-    public Map<String, String> getHeaders() {
-        return Collections.unmodifiableMap(headers);
-    }
-
-    public void addHeader(final String key, final String value) {
-        if (key.contains(" ")) {
-            throw new IllegalArgumentException("Header key may not contain spaces.");
-        } else if ("content-length".equalsIgnoreCase(key)) {
-            throw new IllegalArgumentException("Content-Length header is set automatically based on length of content.");
-        }
-        headers.put(key, value);
-    }
-
-    public void addHeaders(final Map<String, String> headers) {
-        for (final Map.Entry<String, String> entry : headers.entrySet()) {
-            addHeader(entry.getKey(), entry.getValue());
-        }
-    }
-
-    @Override
-    public String toString() {
-
-        final StringBuilder strb = new StringBuilder();
-
-        // response line
-        strb.append("HTTP/1.1 ")
-                .append(status.getStatusCode())
-                .append(" ")
-                .append(status.getReasonPhrase())
-                .append("\n");
-
-        // headers
-        for (final Map.Entry<String, String> entry : headers.entrySet()) {
-            strb.append(entry.getKey()).append(": ").append(entry.getValue()).append("\n");
-        }
-
-        strb.append("\n");
-
-        // body
-        strb.append(entity);
-
-        return strb.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponseAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponseAction.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponseAction.java
deleted file mode 100644
index 28615d0..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponseAction.java
+++ /dev/null
@@ -1,60 +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.cluster.manager.testutils;
-
-/**
- * Wraps a HttpResponse with a time-delay. When the action is applied, the
- * currently executing thread sleeps for the given delay before returning the
- * response to the caller.
- *
- * This class is good for simulating network latency.
- *
- * @author unattributed
- */
-public class HttpResponseAction {
-
-    private final HttpResponse response;
-
-    private final int waitTimeMs;
-
-    public HttpResponseAction(final HttpResponse response) {
-        this(response, 0);
-    }
-
-    public HttpResponseAction(final HttpResponse response, final int waitTimeMs) {
-        this.response = response;
-        this.waitTimeMs = waitTimeMs;
-    }
-
-    public HttpResponse apply() {
-        try {
-            Thread.sleep(waitTimeMs);
-        } catch (final InterruptedException ie) {
-            throw new RuntimeException(ie);
-        }
-
-        return response;
-    }
-
-    public HttpResponse getResponse() {
-        return response;
-    }
-
-    public int getWaitTimeMs() {
-        return waitTimeMs;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpServer.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpServer.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpServer.java
deleted file mode 100644
index f17a66c..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpServer.java
+++ /dev/null
@@ -1,240 +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.cluster.manager.testutils;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.PrintWriter;
-import java.io.Reader;
-import java.net.ServerSocket;
-import java.net.Socket;
-import java.net.SocketException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import org.apache.nifi.cluster.manager.testutils.HttpRequest.HttpRequestBuilder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A simple HTTP web server that allows clients to register canned-responses to
- * respond to received requests.
- *
- * @author unattributed
- */
-public class HttpServer {
-
-    private static final Logger logger = LoggerFactory.getLogger(HttpServer.class);
-
-    private final ExecutorService executorService;
-    private final ServerSocket serverSocket;
-    private final Queue<HttpResponseAction> responseQueue = new ConcurrentLinkedQueue<>();
-    private final Map<String, String> checkedHeaders = new HashMap<>();
-    private final Map<String, List<String>> checkedParameters = new HashMap<>();
-    private final int port;
-
-    public HttpServer(int numThreads, int port) throws IOException {
-        this.port = port;
-        executorService = Executors.newFixedThreadPool(numThreads);
-        serverSocket = new ServerSocket(port);
-    }
-
-    public void start() {
-
-        new Thread() {
-            @Override
-            public void run() {
-                while (isRunning()) {
-                    try {
-                        final Socket conn = serverSocket.accept();
-                        executorService.execute(new Runnable() {
-                            @Override
-                            public void run() {
-                                handleRequest(conn);
-                                if (conn.isClosed() == false) {
-                                    try {
-                                        conn.close();
-                                    } catch (IOException ioe) {
-                                    }
-                                }
-                            }
-                        });
-                    } catch (final SocketException se) {
-                        /* ignored */
-                    } catch (final IOException ioe) {
-                        if (logger.isDebugEnabled()) {
-                            logger.warn("", ioe);
-                        }
-                    }
-                }
-            }
-        ;
-    }
-
-    .start();
-    }
-
-    public boolean isRunning() {
-        return executorService.isShutdown() == false;
-    }
-
-    public void stop() {
-        // shutdown server socket
-        try {
-            if (serverSocket.isClosed() == false) {
-                serverSocket.close();
-            }
-        } catch (final Exception ex) {
-            throw new RuntimeException(ex);
-        }
-
-        // shutdown executor service
-        try {
-            executorService.shutdown();
-            executorService.awaitTermination(3, TimeUnit.SECONDS);
-        } catch (final Exception ex) {
-            throw new RuntimeException(ex);
-        }
-    }
-
-    public int getPort() {
-        if (isRunning()) {
-            return serverSocket.getLocalPort();
-        } else {
-            return port;
-        }
-    }
-
-    public Queue<HttpResponseAction> addResponseAction(final HttpResponseAction response) {
-        responseQueue.add(response);
-        return responseQueue;
-    }
-
-    public void addCheckedHeaders(final Map<String, String> headers) {
-        checkedHeaders.putAll(headers);
-    }
-
-    public void addCheckedParameters(final Map<String, List<String>> parameters) {
-        checkedParameters.putAll(parameters);
-    }
-
-    private void handleRequest(final Socket conn) {
-        try {
-
-            final HttpRequest httpRequest = buildRequest(conn.getInputStream());
-
-            if (logger.isDebugEnabled()) {
-                logger.debug("\n" + httpRequest);
-            }
-
-            // check headers
-            final Map<String, String> reqHeaders = httpRequest.getHeaders();
-            for (final Map.Entry<String, String> entry : checkedHeaders.entrySet()) {
-                if (reqHeaders.containsKey(entry.getKey())) {
-                    if (entry.getValue().equals(reqHeaders.get(entry.getKey()))) {
-                        logger.error("Incorrect HTTP request header value received for checked header: " + entry.getKey());
-                        conn.close();
-                        return;
-                    }
-                } else {
-                    logger.error("Missing checked header: " + entry.getKey());
-                    conn.close();
-                    return;
-                }
-            }
-
-            // check parameters
-            final Map<String, List<String>> reqParams = httpRequest.getParameters();
-            for (final Map.Entry<String, List<String>> entry : checkedParameters.entrySet()) {
-                if (reqParams.containsKey(entry.getKey())) {
-                    if (entry.getValue().equals(reqParams.get(entry.getKey())) == false) {
-                        logger.error("Incorrect HTTP request parameter values received for checked parameter: " + entry.getKey());
-                        conn.close();
-                        return;
-                    }
-                } else {
-                    logger.error("Missing checked parameter: " + entry.getKey());
-                    conn.close();
-                    return;
-                }
-            }
-
-            // apply the next response
-            final HttpResponseAction response = responseQueue.remove();
-            response.apply();
-
-            // send the response to client
-            final PrintWriter pw = new PrintWriter(conn.getOutputStream(), true);
-
-            if (logger.isDebugEnabled()) {
-                logger.debug("\n" + response.getResponse());
-            }
-
-            pw.print(response.getResponse());
-            pw.flush();
-
-        } catch (IOException ioe) { /* ignored */ }
-    }
-
-    private HttpRequest buildRequest(final InputStream requestIs) throws IOException {
-        return new HttpRequestReader().read(new InputStreamReader(requestIs));
-    }
-
-    // reads an HTTP request from the given reader
-    private class HttpRequestReader {
-
-        public HttpRequest read(final Reader reader) throws IOException {
-
-            HttpRequestBuilder builder = null;
-            String line = "";
-            boolean isRequestLine = true;
-            while ((line = readLine(reader)).isEmpty() == false) {
-                if (isRequestLine) {
-                    builder = HttpRequest.createFromRequestLine(line);
-                    isRequestLine = false;
-                } else {
-                    builder.addHeader(line);
-                }
-            }
-
-            if (builder != null) {
-                builder.addBody(reader);
-            }
-
-            return builder.build();
-        }
-
-        private String readLine(final Reader reader) throws IOException {
-
-            /* read character at time to prevent blocking */
-            final StringBuilder strb = new StringBuilder();
-            char c;
-            while ((c = (char) reader.read()) != '\n') {
-                if (c != '\r') {
-                    strb.append(c);
-                }
-            }
-            return strb.toString();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java
deleted file mode 100644
index 96943c2..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java
+++ /dev/null
@@ -1,133 +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.cluster.protocol.impl;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
-import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
-import org.apache.nifi.cluster.protocol.message.PingMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.io.socket.ServerSocketConfiguration;
-import org.apache.nifi.io.socket.SocketConfiguration;
-import org.junit.After;
-import static org.junit.Assert.*;
-import org.junit.Before;
-import org.junit.Test;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.*;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * @author unattributed
- */
-public class ClusterManagerProtocolSenderImplTest {
-
-    private InetAddress address;
-
-    private int port;
-
-    private SocketProtocolListener listener;
-
-    private ClusterManagerProtocolSenderImpl sender;
-
-    private ProtocolHandler mockHandler;
-
-    @Before
-    public void setup() throws IOException {
-
-        address = InetAddress.getLocalHost();
-        ServerSocketConfiguration serverSocketConfiguration = new ServerSocketConfiguration();
-
-        mockHandler = mock(ProtocolHandler.class);
-
-        ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
-
-        listener = new SocketProtocolListener(5, 0, serverSocketConfiguration, protocolContext);
-        listener.addHandler(mockHandler);
-        listener.start();
-
-        port = listener.getPort();
-
-        SocketConfiguration socketConfiguration = new SocketConfiguration();
-        sender = new ClusterManagerProtocolSenderImpl(socketConfiguration, protocolContext);
-    }
-
-    @After
-    public void teardown() throws IOException {
-        if (listener.isRunning()) {
-            listener.stop();
-        }
-    }
-
-    @Test
-    public void testRequestFlow() throws Exception {
-
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new FlowResponseMessage());
-        FlowRequestMessage request = new FlowRequestMessage();
-        request.setNodeId(new NodeIdentifier("id", "api-address", 1, address.getHostAddress(), port));
-        FlowResponseMessage response = sender.requestFlow(request);
-        assertNotNull(response);
-    }
-
-    @Test
-    public void testRequestFlowWithBadResponseMessage() throws Exception {
-
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new PingMessage());
-        FlowRequestMessage request = new FlowRequestMessage();
-        request.setNodeId(new NodeIdentifier("id", "api-address", 1, address.getHostAddress(), port));
-        try {
-            sender.requestFlow(request);
-            fail("failed to throw exception");
-        } catch (ProtocolException pe) {
-        }
-
-    }
-
-    @Test
-    public void testRequestFlowDelayedResponse() throws Exception {
-
-        final int time = 250;
-        sender.getSocketConfiguration().setSocketTimeout(time);
-
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenAnswer(new Answer<FlowResponseMessage>() {
-            @Override
-            public FlowResponseMessage answer(InvocationOnMock invocation) throws Throwable {
-                Thread.sleep(time * 3);
-                return new FlowResponseMessage();
-            }
-        });
-        FlowRequestMessage request = new FlowRequestMessage();
-        request.setNodeId(new NodeIdentifier("id", "api-address", 1, address.getHostAddress(), port));
-        try {
-            sender.requestFlow(request);
-            fail("failed to throw exception");
-        } catch (ProtocolException pe) {
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java
deleted file mode 100644
index 4a69571..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java
+++ /dev/null
@@ -1,119 +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.cluster.protocol.impl;
-
-import java.net.InetSocketAddress;
-import java.util.concurrent.TimeUnit;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-import static org.junit.Assert.*;
-import org.junit.Before;
-import org.junit.Test;
-import static org.mockito.Mockito.*;
-import org.mockito.stubbing.OngoingStubbing;
-
-public class ClusterServiceLocatorTest {
-
-    private ClusterServiceDiscovery mockServiceDiscovery;
-
-    private int fixedPort;
-
-    private DiscoverableService fixedService;
-
-    private ClusterServiceLocator serviceDiscoveryLocator;
-
-    private ClusterServiceLocator serviceDiscoveryFixedPortLocator;
-
-    private ClusterServiceLocator fixedServiceLocator;
-
-    @Before
-    public void setup() throws Exception {
-
-        fixedPort = 1;
-        mockServiceDiscovery = mock(ClusterServiceDiscovery.class);
-        fixedService = new DiscoverableServiceImpl("some-service", InetSocketAddress.createUnresolved("some-host", 20));
-
-        serviceDiscoveryLocator = new ClusterServiceLocator(mockServiceDiscovery);
-        serviceDiscoveryFixedPortLocator = new ClusterServiceLocator(mockServiceDiscovery, fixedPort);
-        fixedServiceLocator = new ClusterServiceLocator(fixedService);
-
-    }
-
-    @Test
-    public void getServiceWhenServiceDiscoveryNotStarted() {
-        assertNull(serviceDiscoveryLocator.getService());
-    }
-
-    @Test
-    public void getServiceWhenServiceDiscoveryFixedPortNotStarted() {
-        assertNull(serviceDiscoveryLocator.getService());
-    }
-
-    @Test
-    public void getServiceWhenFixedServiceNotStarted() {
-        assertEquals(fixedService, fixedServiceLocator.getService());
-    }
-
-    @Test
-    public void getServiceNotOnFirstAttempt() {
-
-        ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig();
-        config.setNumAttempts(2);
-        config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS);
-        config.setTimeBetweenAttempts(1);
-
-        serviceDiscoveryLocator.setAttemptsConfig(config);
-
-        OngoingStubbing<DiscoverableService> stubbing = null;
-        for (int i = 0; i < config.getNumAttempts() - 1; i++) {
-            if (stubbing == null) {
-                stubbing = when(mockServiceDiscovery.getService()).thenReturn(null);
-            } else {
-                stubbing.thenReturn(null);
-            }
-        }
-        stubbing.thenReturn(fixedService);
-
-        assertEquals(fixedService, serviceDiscoveryLocator.getService());
-
-    }
-
-    @Test
-    public void getServiceNotOnFirstAttemptWithFixedPort() {
-
-        ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig();
-        config.setNumAttempts(2);
-        config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS);
-        config.setTimeBetweenAttempts(1);
-
-        serviceDiscoveryFixedPortLocator.setAttemptsConfig(config);
-
-        OngoingStubbing<DiscoverableService> stubbing = null;
-        for (int i = 0; i < config.getNumAttempts() - 1; i++) {
-            if (stubbing == null) {
-                stubbing = when(mockServiceDiscovery.getService()).thenReturn(null);
-            } else {
-                stubbing.thenReturn(null);
-            }
-        }
-        stubbing.thenReturn(fixedService);
-
-        InetSocketAddress resultAddress = InetSocketAddress.createUnresolved(fixedService.getServiceAddress().getHostName(), fixedPort);
-        DiscoverableService resultService = new DiscoverableServiceImpl(fixedService.getServiceName(), resultAddress);
-        assertEquals(resultService, serviceDiscoveryFixedPortLocator.getService());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java
deleted file mode 100644
index 4d85d1a..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java
+++ /dev/null
@@ -1,131 +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.cluster.protocol.impl;
-
-import java.net.InetSocketAddress;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.junit.After;
-import static org.junit.Assert.*;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-/**
- * @author unattributed
- */
-public class ClusterServicesBroadcasterTest {
-
-    private ClusterServicesBroadcaster broadcaster;
-
-    private MulticastProtocolListener listener;
-
-    private DummyProtocolHandler handler;
-
-    private InetSocketAddress multicastAddress;
-
-    private DiscoverableService broadcastedService;
-
-    private ProtocolContext protocolContext;
-
-    private MulticastConfiguration configuration;
-
-    @Before
-    public void setup() throws Exception {
-
-        broadcastedService = new DiscoverableServiceImpl("some-service", new InetSocketAddress("localhost", 11111));
-
-        multicastAddress = new InetSocketAddress("225.1.1.1", 22222);
-
-        configuration = new MulticastConfiguration();
-
-        protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
-
-        broadcaster = new ClusterServicesBroadcaster(multicastAddress, configuration, protocolContext, "500 ms");
-        broadcaster.addService(broadcastedService);
-
-        handler = new DummyProtocolHandler();
-        listener = new MulticastProtocolListener(5, multicastAddress, configuration, protocolContext);
-        listener.addHandler(handler);
-    }
-
-    @After
-    public void teardown() {
-
-        if (broadcaster.isRunning()) {
-            broadcaster.stop();
-        }
-
-        try {
-            if (listener.isRunning()) {
-                listener.stop();
-            }
-        } catch (Exception ex) {
-            ex.printStackTrace(System.out);
-        }
-
-    }
-
-    @Test
-    @Ignore
-    public void testBroadcastReceived() throws Exception {
-
-        broadcaster.start();
-        listener.start();
-
-        Thread.sleep(1000);
-
-        listener.stop();
-
-        assertNotNull(handler.getProtocolMessage());
-        assertEquals(ProtocolMessage.MessageType.SERVICE_BROADCAST, handler.getProtocolMessage().getType());
-        final ServiceBroadcastMessage msg = (ServiceBroadcastMessage) handler.getProtocolMessage();
-        assertEquals(broadcastedService.getServiceName(), msg.getServiceName());
-        assertEquals(broadcastedService.getServiceAddress().getHostName(), msg.getAddress());
-        assertEquals(broadcastedService.getServiceAddress().getPort(), msg.getPort());
-    }
-
-    private class DummyProtocolHandler implements ProtocolHandler {
-
-        private ProtocolMessage protocolMessage;
-
-        @Override
-        public boolean canHandle(ProtocolMessage msg) {
-            return true;
-        }
-
-        @Override
-        public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
-            this.protocolMessage = msg;
-            return null;
-        }
-
-        public ProtocolMessage getProtocolMessage() {
-            return protocolMessage;
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java
deleted file mode 100644
index 6c79b90..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java
+++ /dev/null
@@ -1,171 +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.cluster.protocol.impl;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.net.DatagramPacket;
-import java.net.InetSocketAddress;
-import java.net.MulticastSocket;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.MulticastProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.PingMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.apache.nifi.io.socket.multicast.MulticastUtils;
-import org.junit.After;
-import static org.junit.Assert.*;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-/**
- * @author unattributed
- */
-public class MulticastProtocolListenerTest {
-
-    private MulticastProtocolListener listener;
-
-    private MulticastSocket socket;
-
-    private InetSocketAddress address;
-
-    private MulticastConfiguration configuration;
-
-    private ProtocolContext protocolContext;
-
-    @Before
-    public void setup() throws Exception {
-
-        address = new InetSocketAddress("226.1.1.1", 60000);
-        configuration = new MulticastConfiguration();
-
-        protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
-
-        listener = new MulticastProtocolListener(5, address, configuration, protocolContext);
-        listener.start();
-
-        socket = MulticastUtils.createMulticastSocket(address.getPort(), configuration);
-    }
-
-    @After
-    public void teardown() throws IOException {
-        try {
-            if (listener.isRunning()) {
-                listener.stop();
-            }
-        } finally {
-            MulticastUtils.closeQuietly(socket);
-        }
-    }
-
-    @Ignore("This test must be reworked.  Requires an active network connection")
-    @Test
-    public void testBadRequest() throws Exception {
-        DelayedProtocolHandler handler = new DelayedProtocolHandler(0);
-        listener.addHandler(handler);
-        DatagramPacket packet = new DatagramPacket(new byte[]{5}, 1, address);
-        socket.send(packet);
-        Thread.sleep(250);
-        assertEquals(0, handler.getMessages().size());
-    }
-
-    @Test
-    @Ignore
-    public void testRequest() throws Exception {
-
-        ReflexiveProtocolHandler handler = new ReflexiveProtocolHandler();
-        listener.addHandler(handler);
-
-        ProtocolMessage msg = new PingMessage();
-        MulticastProtocolMessage multicastMsg = new MulticastProtocolMessage("some-id", msg);
-
-        // marshal message to output stream
-        ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller();
-        ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        marshaller.marshal(multicastMsg, baos);
-        byte[] requestPacketBytes = baos.toByteArray();
-        DatagramPacket packet = new DatagramPacket(requestPacketBytes, requestPacketBytes.length, address);
-        socket.send(packet);
-
-        Thread.sleep(250);
-        assertEquals(1, handler.getMessages().size());
-        assertEquals(msg.getType(), handler.getMessages().get(0).getType());
-
-    }
-
-    private class ReflexiveProtocolHandler implements ProtocolHandler {
-
-        private List<ProtocolMessage> messages = new ArrayList<>();
-
-        @Override
-        public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
-            messages.add(msg);
-            return msg;
-        }
-
-        @Override
-        public boolean canHandle(ProtocolMessage msg) {
-            return true;
-        }
-
-        public List<ProtocolMessage> getMessages() {
-            return messages;
-        }
-
-    }
-
-    private class DelayedProtocolHandler implements ProtocolHandler {
-
-        private int delay = 0;
-
-        private List<ProtocolMessage> messages = new ArrayList<>();
-
-        public DelayedProtocolHandler(int delay) {
-            this.delay = delay;
-        }
-
-        @Override
-        public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
-            try {
-                messages.add(msg);
-                Thread.sleep(delay);
-                return null;
-            } catch (final InterruptedException ie) {
-                throw new ProtocolException(ie);
-            }
-
-        }
-
-        @Override
-        public boolean canHandle(ProtocolMessage msg) {
-            return true;
-        }
-
-        public List<ProtocolMessage> getMessages() {
-            return messages;
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java
deleted file mode 100644
index 7c62d2f..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java
+++ /dev/null
@@ -1,201 +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.cluster.protocol.impl;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.UUID;
-
-import org.apache.nifi.cluster.HeartbeatPayload;
-import org.apache.nifi.cluster.protocol.ConnectionRequest;
-import org.apache.nifi.cluster.protocol.ConnectionResponse;
-import org.apache.nifi.cluster.protocol.Heartbeat;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.StandardDataFlow;
-import org.apache.nifi.cluster.protocol.UnknownServiceAddressException;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage;
-import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage;
-import org.apache.nifi.cluster.protocol.message.ControllerStartupFailureMessage;
-import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
-import org.apache.nifi.cluster.protocol.message.PingMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.io.socket.ServerSocketConfiguration;
-import org.apache.nifi.io.socket.SocketConfiguration;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * @author unattributed
- */
-public class NodeProtocolSenderImplTest {
-
-    private SocketProtocolListener listener;
-
-    private NodeProtocolSenderImpl sender;
-
-    private DiscoverableService service;
-
-    private ServerSocketConfiguration serverSocketConfiguration;
-
-    private ClusterServiceLocator mockServiceLocator;
-
-    private ProtocolHandler mockHandler;
-
-    private NodeIdentifier nodeIdentifier;
-
-    @Before
-    public void setup() throws IOException {
-
-        serverSocketConfiguration = new ServerSocketConfiguration();
-
-        mockServiceLocator = mock(ClusterServiceLocator.class);
-        mockHandler = mock(ProtocolHandler.class);
-
-        nodeIdentifier = new NodeIdentifier("1", "localhost", 1234, "localhost", 5678);
-
-        ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
-
-        listener = new SocketProtocolListener(5, 0, serverSocketConfiguration, protocolContext);
-        listener.setShutdownListenerSeconds(3);
-        listener.addHandler(mockHandler);
-        listener.start();
-
-        service = new DiscoverableServiceImpl("some-service", new InetSocketAddress("localhost", listener.getPort()));
-
-        SocketConfiguration socketConfiguration = new SocketConfiguration();
-        socketConfiguration.setReuseAddress(true);
-        sender = new NodeProtocolSenderImpl(mockServiceLocator, socketConfiguration, protocolContext);
-    }
-
-    @After
-    public void teardown() throws IOException {
-        if (listener.isRunning()) {
-            listener.stop();
-        }
-    }
-
-    @Test
-    public void testConnect() throws Exception {
-
-        when(mockServiceLocator.getService()).thenReturn(service);
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        ConnectionResponseMessage mockMessage = new ConnectionResponseMessage();
-        mockMessage.setConnectionResponse(new ConnectionResponse(nodeIdentifier, new StandardDataFlow("flow".getBytes("UTF-8"), new byte[0], new byte[0]), false, null, null, UUID.randomUUID().toString()));
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(mockMessage);
-
-        ConnectionRequestMessage request = new ConnectionRequestMessage();
-        request.setConnectionRequest(new ConnectionRequest(nodeIdentifier));
-        ConnectionResponseMessage response = sender.requestConnection(request);
-        assertNotNull(response);
-    }
-
-    @Test(expected = UnknownServiceAddressException.class)
-    public void testConnectNoClusterManagerAddress() throws Exception {
-
-        when(mockServiceLocator.getService()).thenReturn(null);
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new ConnectionResponseMessage());
-
-        ConnectionRequestMessage request = new ConnectionRequestMessage();
-        request.setConnectionRequest(new ConnectionRequest(nodeIdentifier));
-
-        sender.requestConnection(request);
-        fail("failed to throw exception");
-    }
-
-    @Test(expected = ProtocolException.class)
-    public void testConnectBadResponse() throws Exception {
-
-        when(mockServiceLocator.getService()).thenReturn(service);
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new PingMessage());
-
-        ConnectionRequestMessage request = new ConnectionRequestMessage();
-        request.setConnectionRequest(new ConnectionRequest(nodeIdentifier));
-
-        sender.requestConnection(request);
-        fail("failed to throw exception");
-
-    }
-
-    @Test(expected = ProtocolException.class)
-    public void testConnectDelayedResponse() throws Exception {
-
-        final int time = 250;
-        sender.getSocketConfiguration().setSocketTimeout(time);
-        when(mockServiceLocator.getService()).thenReturn(service);
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenAnswer(new Answer<ConnectionResponseMessage>() {
-            @Override
-            public ConnectionResponseMessage answer(InvocationOnMock invocation) throws Throwable {
-                Thread.sleep(time * 3);
-                return new ConnectionResponseMessage();
-            }
-        });
-        ConnectionRequestMessage request = new ConnectionRequestMessage();
-        request.setConnectionRequest(new ConnectionRequest(nodeIdentifier));
-
-        sender.requestConnection(request);
-        fail("failed to throw exception");
-
-    }
-
-    @Test
-    public void testHeartbeat() throws Exception {
-
-        when(mockServiceLocator.getService()).thenReturn(service);
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(null);
-
-        HeartbeatMessage msg = new HeartbeatMessage();
-        HeartbeatPayload hbPayload = new HeartbeatPayload();
-        Heartbeat hb = new Heartbeat(new NodeIdentifier("id", "localhost", 3, "localhost", 4), false, false, hbPayload.marshal());
-        msg.setHeartbeat(hb);
-        sender.heartbeat(msg);
-    }
-
-    @Test
-    public void testNotifyControllerStartupFailure() throws Exception {
-
-        when(mockServiceLocator.getService()).thenReturn(service);
-        when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
-        when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(null);
-
-        ControllerStartupFailureMessage msg = new ControllerStartupFailureMessage();
-        msg.setNodeId(new NodeIdentifier("some-id", "some-addr", 1, "some-addr", 1));
-        msg.setExceptionMessage("some exception");
-        sender.notifyControllerStartupFailure(msg);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListenerTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListenerTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListenerTest.java
deleted file mode 100644
index 92a7d2a..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListenerTest.java
+++ /dev/null
@@ -1,132 +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.cluster.protocol.impl;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.net.SocketTimeoutException;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.PingMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.cluster.protocol.testutils.DelayedProtocolHandler;
-import org.apache.nifi.cluster.protocol.testutils.ReflexiveProtocolHandler;
-import org.apache.nifi.io.socket.ServerSocketConfiguration;
-import org.apache.nifi.io.socket.SocketConfiguration;
-import org.apache.nifi.io.socket.SocketUtils;
-import org.junit.After;
-import static org.junit.Assert.*;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * @author unattributed
- */
-public class SocketProtocolListenerTest {
-
-    private SocketProtocolListener listener;
-
-    private Socket socket;
-
-    private ProtocolMessageMarshaller<ProtocolMessage> marshaller;
-
-    private ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller;
-
-    @Before
-    public void setup() throws Exception {
-
-        final ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
-        marshaller = protocolContext.createMarshaller();
-        unmarshaller = protocolContext.createUnmarshaller();
-
-        ServerSocketConfiguration configuration = new ServerSocketConfiguration();
-        configuration.setSocketTimeout(1000);
-
-        listener = new SocketProtocolListener(5, 0, configuration, protocolContext);
-        listener.start();
-
-        int port = listener.getPort();
-
-        SocketConfiguration config = new SocketConfiguration();
-        config.setReuseAddress(true);
-        config.setSocketTimeout(1000);
-        socket = SocketUtils.createSocket(new InetSocketAddress("localhost", port), config);
-    }
-
-    @After
-    public void teardown() throws IOException {
-        try {
-            if (listener.isRunning()) {
-                listener.stop();
-            }
-        } finally {
-            SocketUtils.closeQuietly(socket);
-        }
-    }
-
-    @Test
-    public void testBadRequest() throws Exception {
-        DelayedProtocolHandler handler = new DelayedProtocolHandler(0);
-        listener.addHandler(handler);
-        socket.getOutputStream().write(5);
-        Thread.sleep(250);
-        assertEquals(0, handler.getMessages().size());
-    }
-
-    @Test
-    public void testRequest() throws Exception {
-        ProtocolMessage msg = new PingMessage();
-
-        ReflexiveProtocolHandler handler = new ReflexiveProtocolHandler();
-        listener.addHandler(handler);
-
-        // marshal message to output stream
-        marshaller.marshal(msg, socket.getOutputStream());
-
-        // unmarshall response and return
-        ProtocolMessage response = unmarshaller.unmarshal(socket.getInputStream());
-        assertEquals(msg.getType(), response.getType());
-
-        assertEquals(1, handler.getMessages().size());
-        assertEquals(msg.getType(), handler.getMessages().get(0).getType());
-    }
-
-    @Test
-    public void testDelayedRequest() throws Exception {
-        ProtocolMessage msg = new PingMessage();
-
-        DelayedProtocolHandler handler = new DelayedProtocolHandler(2000);
-        listener.addHandler(handler);
-
-        // marshal message to output stream
-        marshaller.marshal(msg, socket.getOutputStream());
-
-        try {
-            socket.getInputStream().read();
-            fail("Socket timeout not received.");
-        } catch (SocketTimeoutException ste) {
-        }
-
-        assertEquals(1, handler.getMessages().size());
-        assertEquals(msg.getType(), handler.getMessages().get(0).getType());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/DelayedProtocolHandler.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/DelayedProtocolHandler.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/DelayedProtocolHandler.java
deleted file mode 100644
index 2f16777..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/DelayedProtocolHandler.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.cluster.protocol.testutils;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-
-/**
- * @author unattributed
- */
-public class DelayedProtocolHandler implements ProtocolHandler {
-
-    private int delay = 0;
-    private List<ProtocolMessage> messages = new ArrayList<>();
-
-    public DelayedProtocolHandler(int delay) {
-        this.delay = delay;
-    }
-
-    @Override
-    public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
-        try {
-            messages.add(msg);
-            Thread.sleep(delay);
-            return null;
-        } catch (final InterruptedException ie) {
-            throw new ProtocolException(ie);
-        }
-
-    }
-
-    @Override
-    public boolean canHandle(ProtocolMessage msg) {
-        return true;
-    }
-
-    public List<ProtocolMessage> getMessages() {
-        return messages;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/ReflexiveProtocolHandler.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/ReflexiveProtocolHandler.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/ReflexiveProtocolHandler.java
deleted file mode 100644
index e80f52c..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/ReflexiveProtocolHandler.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.cluster.protocol.testutils;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-
-/**
- * @author unattributed
- */
-public class ReflexiveProtocolHandler implements ProtocolHandler {
-
-    private List<ProtocolMessage> messages = new ArrayList<>();
-
-    @Override
-    public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
-        messages.add(msg);
-        return msg;
-    }
-
-    @Override
-    public boolean canHandle(ProtocolMessage msg) {
-        return true;
-    }
-
-    public List<ProtocolMessage> getMessages() {
-        return messages;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/resources/logback-test.xml b/nar-bundles/framework-bundle/framework/cluster/src/test/resources/logback-test.xml
deleted file mode 100644
index 92eb78c..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/resources/logback-test.xml
+++ /dev/null
@@ -1,48 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
-<configuration scan="true" scanPeriod="30 seconds">
-    <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender">
-        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
-            <pattern>%-4r [%t] %-5p %c - %m%n</pattern>
-        </encoder>
-    </appender>
-    
-    <!-- valid logging levels: TRACE, DEBUG, INFO, WARN, ERROR -->
-    <logger name="org.apache.nifi" level="INFO"/>
-    
-    <!-- Logger for managing logging statements for nifi clusters. -->
-    <logger name="org.apache.nifi.cluster" level="INFO"/>
-
-    <!-- 
-        Logger for logging HTTP requests received by the web server.  Setting
-        log level to 'debug' activates HTTP request logging.
-    -->
-    <logger name="org.apache.nifi.server.JettyServer" level="INFO"/>
-
-    <!-- Logger for managing logging statements for jetty -->
-    <logger name="org.mortbay" level="INFO"/>
-
-    <!-- Suppress non-error messages due to excessive logging by class -->
-    <logger name="com.sun.jersey.spi.container.servlet.WebComponent" level="ERROR"/>
-
-    <logger name="org.apache.nifi.processors.standard" level="DEBUG"/>
-
-    <root level="INFO">
-        <appender-ref ref="CONSOLE"/>
-    </root>
-    
-</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/empty.txt
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/empty.txt b/nar-bundles/framework-bundle/framework/cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/empty.txt
deleted file mode 100755
index e69de29..0000000

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/ips.txt
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/ips.txt b/nar-bundles/framework-bundle/framework/cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/ips.txt
deleted file mode 100755
index e8e4c2b..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/ips.txt
+++ /dev/null
@@ -1,12 +0,0 @@
-
-bad data should be skipped
-
-# this is a comment
-  2.2.2.2  # this is another comment ####
-3.3.3.3/8
-
-4.4.4.4/24
-
-5.5.5.255/31
-
-more bad data
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/.gitignore
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/.gitignore b/nar-bundles/framework-bundle/framework/core-api/.gitignore
deleted file mode 100755
index ea8c4bf..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/.gitignore
+++ /dev/null
@@ -1 +0,0 @@
-/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/pom.xml b/nar-bundles/framework-bundle/framework/core-api/pom.xml
deleted file mode 100644
index 63645f3..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/pom.xml
+++ /dev/null
@@ -1,56 +0,0 @@
-<?xml version="1.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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-framework-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-    <artifactId>core-api</artifactId>
-    <version>0.0.1-SNAPSHOT</version>
-    <name>NiFi Core API</name>
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-nar</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-runtime</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>client-dto</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-lang3</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.quartz-scheduler</groupId>
-            <artifactId>quartz</artifactId>
-        </dependency>
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java
deleted file mode 100644
index 0092f7a..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java
+++ /dev/null
@@ -1,66 +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.cluster;
-
-public class AdaptedNodeInformation {
-
-    private String hostname;
-    private Integer siteToSitePort;
-    private int apiPort;
-    private boolean isSiteToSiteSecure;
-    private int totalFlowFiles;
-
-    public String getHostname() {
-        return hostname;
-    }
-
-    public void setHostname(String hostname) {
-        this.hostname = hostname;
-    }
-
-    public Integer getSiteToSitePort() {
-        return siteToSitePort;
-    }
-
-    public void setSiteToSitePort(Integer siteToSitePort) {
-        this.siteToSitePort = siteToSitePort;
-    }
-
-    public int getApiPort() {
-        return apiPort;
-    }
-
-    public void setApiPort(int apiPort) {
-        this.apiPort = apiPort;
-    }
-
-    public boolean isSiteToSiteSecure() {
-        return isSiteToSiteSecure;
-    }
-
-    public void setSiteToSiteSecure(boolean isSiteToSiteSecure) {
-        this.isSiteToSiteSecure = isSiteToSiteSecure;
-    }
-
-    public int getTotalFlowFiles() {
-        return totalFlowFiles;
-    }
-
-    public void setTotalFlowFiles(int totalFlowFiles) {
-        this.totalFlowFiles = totalFlowFiles;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java
deleted file mode 100644
index 5751c32..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java
+++ /dev/null
@@ -1,67 +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.cluster;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Collection;
-
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-
-@XmlRootElement
-public class ClusterNodeInformation {
-
-    private Collection<NodeInformation> nodeInfo;
-
-    private static final JAXBContext JAXB_CONTEXT;
-
-    static {
-        try {
-            JAXB_CONTEXT = JAXBContext.newInstance(ClusterNodeInformation.class);
-        } catch (JAXBException e) {
-            throw new RuntimeException("Unable to create JAXBContext.", e);
-        }
-    }
-
-    public ClusterNodeInformation() {
-        this.nodeInfo = null;
-    }
-
-    public void setNodeInformation(final Collection<NodeInformation> nodeInfo) {
-        this.nodeInfo = nodeInfo;
-    }
-
-    @XmlJavaTypeAdapter(NodeInformationAdapter.class)
-    public Collection<NodeInformation> getNodeInformation() {
-        return nodeInfo;
-    }
-
-    public void marshal(final OutputStream os) throws JAXBException {
-        final Marshaller marshaller = JAXB_CONTEXT.createMarshaller();
-        marshaller.marshal(this, os);
-    }
-
-    public static ClusterNodeInformation unmarshal(final InputStream is) throws JAXBException {
-        final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
-        return (ClusterNodeInformation) unmarshaller.unmarshal(is);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java
deleted file mode 100644
index 987ff65..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java
+++ /dev/null
@@ -1,22 +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.cluster;
-
-public interface NodeInformant {
-
-    ClusterNodeInformation getNodeInformation();
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java
deleted file mode 100644
index 848eb7e..0000000
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java
+++ /dev/null
@@ -1,98 +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.cluster;
-
-public class NodeInformation {
-
-    private final String hostname;
-    private final Integer siteToSitePort;
-    private final int apiPort;
-    private final boolean isSiteToSiteSecure;
-    private final int totalFlowFiles;
-
-    public NodeInformation(final String hostname, final Integer siteToSitePort, final int apiPort,
-            final boolean isSiteToSiteSecure, final int totalFlowFiles) {
-        this.hostname = hostname;
-        this.siteToSitePort = siteToSitePort;
-        this.apiPort = apiPort;
-        this.isSiteToSiteSecure = isSiteToSiteSecure;
-        this.totalFlowFiles = totalFlowFiles;
-    }
-
-    public String getHostname() {
-        return hostname;
-    }
-
-    public int getAPIPort() {
-        return apiPort;
-    }
-
-    public Integer getSiteToSitePort() {
-        return siteToSitePort;
-    }
-
-    public boolean isSiteToSiteSecure() {
-        return isSiteToSiteSecure;
-    }
-
-    public int getTotalFlowFiles() {
-        return totalFlowFiles;
-    }
-
-    @Override
-    public boolean equals(final Object obj) {
-        if (obj == this) {
-            return true;
-        }
-        if (obj == null) {
-            return false;
-        }
-        if (!(obj instanceof NodeInformation)) {
-            return false;
-        }
-
-        final NodeInformation other = (NodeInformation) obj;
-        if (!hostname.equals(other.hostname)) {
-            return false;
-        }
-        if (siteToSitePort == null && other.siteToSitePort != null) {
-            return false;
-        }
-        if (siteToSitePort != null && other.siteToSitePort == null) {
-            return false;
-        } else if (siteToSitePort != null && siteToSitePort.intValue() != other.siteToSitePort.intValue()) {
-            return false;
-        }
-        if (apiPort != other.apiPort) {
-            return false;
-        }
-        if (isSiteToSiteSecure != other.isSiteToSiteSecure) {
-            return false;
-        }
-        return true;
-    }
-
-    @Override
-    public int hashCode() {
-        return 83832 + hostname.hashCode() + (siteToSitePort == null ? 8 : siteToSitePort.hashCode()) + apiPort + (isSiteToSiteSecure ? 3829 : 0);
-    }
-
-    @Override
-    public String toString() {
-        return "Node[" + hostname + ":" + apiPort + "]";
-    }
-}


[02/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
deleted file mode 100644
index b60d187..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
+++ /dev/null
@@ -1,1026 +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;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URL;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.StandardOpenOption;
-import java.security.NoSuchAlgorithmException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.zip.GZIPInputStream;
-
-import javax.xml.XMLConstants;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-
-import org.apache.nifi.cluster.protocol.DataFlow;
-import org.apache.nifi.cluster.protocol.StandardDataFlow;
-import org.apache.nifi.connectable.Connectable;
-import org.apache.nifi.connectable.ConnectableType;
-import org.apache.nifi.connectable.Connection;
-import org.apache.nifi.connectable.Funnel;
-import org.apache.nifi.connectable.Port;
-import org.apache.nifi.connectable.Position;
-import org.apache.nifi.connectable.Size;
-import org.apache.nifi.controller.exception.ProcessorInstantiationException;
-import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.events.BulletinFactory;
-import org.apache.nifi.util.file.FileUtils;
-import org.apache.nifi.fingerprint.FingerprintException;
-import org.apache.nifi.fingerprint.FingerprintFactory;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
-import org.apache.nifi.groups.ProcessGroup;
-import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.groups.RemoteProcessGroupPortDescriptor;
-import org.apache.nifi.logging.LogLevel;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.remote.RemoteGroupPort;
-import org.apache.nifi.remote.RootGroupPort;
-import org.apache.nifi.reporting.Severity;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.web.api.dto.ConnectableDTO;
-import org.apache.nifi.web.api.dto.ConnectionDTO;
-import org.apache.nifi.web.api.dto.FlowSnippetDTO;
-import org.apache.nifi.web.api.dto.FunnelDTO;
-import org.apache.nifi.web.api.dto.LabelDTO;
-import org.apache.nifi.web.api.dto.PortDTO;
-import org.apache.nifi.web.api.dto.PositionDTO;
-import org.apache.nifi.web.api.dto.ProcessGroupDTO;
-import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
-import org.apache.nifi.web.api.dto.ProcessorDTO;
-import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
-
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.nifi.encrypt.StringEncryptor;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
-import org.xml.sax.SAXException;
-
-/**
- * @author unattributed
- */
-public class StandardFlowSynchronizer implements FlowSynchronizer {
-
-    private static final Logger logger = LoggerFactory.getLogger(StandardFlowSynchronizer.class);
-    public static final URL FLOW_XSD_RESOURCE = StandardFlowSynchronizer.class.getResource("/FlowConfiguration.xsd");
-    private final StringEncryptor encryptor;
-
-    public StandardFlowSynchronizer(final StringEncryptor encryptor) {
-        this.encryptor = encryptor;
-    }
-
-    public static boolean isEmpty(final DataFlow dataFlow, final StringEncryptor encryptor) {
-        if (dataFlow == null || dataFlow.getFlow() == null || dataFlow.getFlow().length == 0) {
-            return true;
-        }
-
-        final Document document = parseFlowBytes(dataFlow.getFlow());
-        final Element rootElement = document.getDocumentElement();
-
-        final Element rootGroupElement = (Element) rootElement.getElementsByTagName("rootGroup").item(0);
-        final ProcessGroupDTO rootGroupDto = FlowFromDOMFactory.getProcessGroup(null, rootGroupElement, encryptor);
-        return isEmpty(rootGroupDto);
-    }
-
-    @Override
-    public void sync(final FlowController controller, final DataFlow proposedFlow, final StringEncryptor encryptor) throws FlowSerializationException, UninheritableFlowException, FlowSynchronizationException {
-        // get the controller's root group
-        final ProcessGroup rootGroup = controller.getGroup(controller.getRootGroupId());
-
-        // handle corner cases involving no proposed flow
-        if (proposedFlow == null) {
-            if (rootGroup.isEmpty()) {
-                return;  // no sync to perform
-            } else {
-                throw new UninheritableFlowException("Proposed configuration is empty, but the controller contains a data flow.");
-            }
-        }
-
-        // determine if the controller has been initialized
-        final boolean initialized = controller.isInitialized();
-        logger.debug("Synching FlowController with proposed flow: Controller is Initialized = {}", initialized);
-
-        // serialize controller state to bytes
-        final byte[] existingFlow;
-        final boolean existingFlowEmpty;
-        try {
-            if (initialized) {
-                existingFlow = toBytes(controller);
-                existingFlowEmpty = controller.getGroup(controller.getRootGroupId()).isEmpty();
-            } else {
-                existingFlow = readFlowFromDisk();
-                if (existingFlow == null || existingFlow.length == 0) {
-                    existingFlowEmpty = true;
-                } else {
-                    final Document document = parseFlowBytes(existingFlow);
-                    final Element rootElement = document.getDocumentElement();
-
-                    logger.trace("Setting controller thread counts");
-                    final Integer maxThreadCount = getInteger(rootElement, "maxThreadCount");
-                    if (maxThreadCount == null) {
-                        controller.setMaxTimerDrivenThreadCount(getInt(rootElement, "maxTimerDrivenThreadCount"));
-                        controller.setMaxEventDrivenThreadCount(getInt(rootElement, "maxEventDrivenThreadCount"));
-                    } else {
-                        controller.setMaxTimerDrivenThreadCount(maxThreadCount * 2 / 3);
-                        controller.setMaxEventDrivenThreadCount(maxThreadCount / 3);
-                    }
-
-                    logger.trace("Parsing process group from DOM");
-                    final Element rootGroupElement = (Element) rootElement.getElementsByTagName("rootGroup").item(0);
-                    final ProcessGroupDTO rootGroupDto = FlowFromDOMFactory.getProcessGroup(null, rootGroupElement, encryptor);
-                    existingFlowEmpty = isEmpty(rootGroupDto);
-                    logger.debug("Existing Flow Empty = {}", existingFlowEmpty);
-                }
-            }
-        } catch (final IOException e) {
-            throw new FlowSerializationException(e);
-        }
-
-        logger.trace("Exporting templates from controller");
-        final byte[] existingTemplates = controller.getTemplateManager().export();
-        logger.trace("Exporting snippets from controller");
-        final byte[] existingSnippets = controller.getSnippetManager().export();
-
-        final DataFlow existingDataFlow = new StandardDataFlow(existingFlow, existingTemplates, existingSnippets);
-
-        final boolean existingTemplatesEmpty = existingTemplates == null || existingTemplates.length == 0;
-
-        // check that the proposed flow is inheritable by the controller
-        try {
-            if (!existingFlowEmpty) {
-                logger.trace("Checking flow inheritability");
-                final String problemInheriting = checkFlowInheritability(existingDataFlow, proposedFlow, controller);
-                if (problemInheriting != null) {
-                    throw new UninheritableFlowException("Proposed configuration is not inheritable by the flow controller because of flow differences: " + problemInheriting);
-                }
-            }
-            if (!existingTemplatesEmpty) {
-                logger.trace("Checking template inheritability");
-                final String problemInheriting = checkTemplateInheritability(existingDataFlow, proposedFlow);
-                if (problemInheriting != null) {
-                    throw new UninheritableFlowException("Proposed configuration is not inheritable by the flow controller because of flow differences: " + problemInheriting);
-                }
-            }
-        } catch (final FingerprintException fe) {
-            throw new FlowSerializationException("Failed to generate flow fingerprints", fe);
-        }
-
-        // create document by parsing proposed flow bytes
-        logger.trace("Parsing proposed flow bytes as DOM document");
-        final Document configuration = parseFlowBytes(proposedFlow.getFlow());
-
-        // attempt to sync controller with proposed flow
-        try {
-            if (configuration != null) {
-                // get the root element
-                final Element rootElement = (Element) configuration.getElementsByTagName("flowController").item(0);
-
-                // set controller config
-                logger.trace("Updating flow config");
-                final Integer maxThreadCount = getInteger(rootElement, "maxThreadCount");
-                if (maxThreadCount == null) {
-                    controller.setMaxTimerDrivenThreadCount(getInt(rootElement, "maxTimerDrivenThreadCount"));
-                    controller.setMaxEventDrivenThreadCount(getInt(rootElement, "maxEventDrivenThreadCount"));
-                } else {
-                    controller.setMaxTimerDrivenThreadCount(maxThreadCount * 2 / 3);
-                    controller.setMaxEventDrivenThreadCount(maxThreadCount / 3);
-                }
-
-                // get the root group XML element
-                final Element rootGroupElement = (Element) rootElement.getElementsByTagName("rootGroup").item(0);
-
-                // if this controller isn't initialized or its emtpy, add the root group, otherwise update
-                if (!initialized || existingFlowEmpty) {
-                    logger.trace("Adding root process group");
-                    addProcessGroup(controller, /* parent group */ null, rootGroupElement, encryptor);
-                } else {
-                    logger.trace("Updating root process group");
-                    updateProcessGroup(controller, /* parent group */ null, rootGroupElement, encryptor);
-                }
-            }
-
-            logger.trace("Synching templates");
-            if ((existingTemplates == null || existingTemplates.length == 0) && proposedFlow.getTemplates() != null && proposedFlow.getTemplates().length > 0) {
-                // need to load templates
-                final TemplateManager templateManager = controller.getTemplateManager();
-                final List<Template> proposedTemplateList = TemplateManager.parseBytes(proposedFlow.getTemplates());
-                for (final Template template : proposedTemplateList) {
-                    templateManager.addTemplate(template.getDetails());
-                }
-            }
-
-            // clear the snippets that are currently in memory
-            logger.trace("Clearing existing snippets");
-            final SnippetManager snippetManager = controller.getSnippetManager();
-            snippetManager.clear();
-
-            // if proposed flow has any snippets, load them
-            logger.trace("Loading proposed snippets");
-            final byte[] proposedSnippets = proposedFlow.getSnippets();
-            if (proposedSnippets != null && proposedSnippets.length > 0) {
-                for (final StandardSnippet snippet : SnippetManager.parseBytes(proposedSnippets)) {
-                    snippetManager.addSnippet(snippet);
-                }
-            }
-
-            logger.debug("Finished synching flows");
-        } catch (final Exception ex) {
-            throw new FlowSynchronizationException(ex);
-        }
-    }
-
-    private static boolean isEmpty(final ProcessGroupDTO dto) {
-        if (dto == null) {
-            return true;
-        }
-
-        final FlowSnippetDTO contents = dto.getContents();
-        if (contents == null) {
-            return true;
-        }
-
-        return CollectionUtils.isEmpty(contents.getProcessors())
-                && CollectionUtils.isEmpty(contents.getConnections())
-                && CollectionUtils.isEmpty(contents.getFunnels())
-                && CollectionUtils.isEmpty(contents.getLabels())
-                && CollectionUtils.isEmpty(contents.getOutputPorts())
-                && CollectionUtils.isEmpty(contents.getProcessGroups())
-                && CollectionUtils.isEmpty(contents.getProcessors())
-                && CollectionUtils.isEmpty(contents.getRemoteProcessGroups());
-    }
-
-    private static Document parseFlowBytes(final byte[] flow) throws FlowSerializationException {
-        // create document by parsing proposed flow bytes
-        try {
-            // create validating document builder
-            final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
-            final Schema schema = schemaFactory.newSchema(FLOW_XSD_RESOURCE);
-            final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance();
-            docFactory.setSchema(schema);
-            final DocumentBuilder docBuilder = docFactory.newDocumentBuilder();
-
-            // parse flow
-            return (flow == null || flow.length == 0) ? null : docBuilder.parse(new ByteArrayInputStream(flow));
-        } catch (final SAXException | ParserConfigurationException | IOException ex) {
-            throw new FlowSerializationException(ex);
-        }
-    }
-
-    private byte[] readFlowFromDisk() throws IOException {
-        final Path flowPath = NiFiProperties.getInstance().getFlowConfigurationFile().toPath();
-        if (!Files.exists(flowPath) || Files.size(flowPath) == 0) {
-            return new byte[0];
-        }
-
-        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-
-        try (final InputStream in = Files.newInputStream(flowPath, StandardOpenOption.READ);
-                final InputStream gzipIn = new GZIPInputStream(in)) {
-            FileUtils.copy(gzipIn, baos);
-        }
-
-        return baos.toByteArray();
-    }
-
-    private ProcessGroup updateProcessGroup(final FlowController controller, final ProcessGroup parentGroup, final Element processGroupElement, final StringEncryptor encryptor) throws ProcessorInstantiationException {
-
-        // get the parent group ID
-        final String parentId = (parentGroup == null) ? null : parentGroup.getIdentifier();
-
-        // get the process group
-        final ProcessGroupDTO processGroupDto = FlowFromDOMFactory.getProcessGroup(parentId, processGroupElement, encryptor);
-
-        // update the process group
-        if (parentId == null) {
-
-            /*
-             * Labels are not included in the "inherit flow" algorithm, so we cannot
-             * blindly update them because they may not exist in the current flow.
-             * Therefore, we first remove all labels, and then let the updating
-             * process add labels defined in the new flow.
-             */
-            final ProcessGroup root = controller.getGroup(controller.getRootGroupId());
-            for (final Label label : root.findAllLabels()) {
-                label.getProcessGroup().removeLabel(label);
-            }
-        }
-
-        // update the process group
-        controller.updateProcessGroup(processGroupDto);
-
-        // get the real process group and ID
-        final ProcessGroup processGroup = controller.getGroup(processGroupDto.getId());
-
-        // processors & ports cannot be updated - they must be the same. Except for the scheduled state.
-        final List<Element> processorNodeList = getChildrenByTagName(processGroupElement, "processor");
-        for (final Element processorElement : processorNodeList) {
-            final ProcessorDTO dto = FlowFromDOMFactory.getProcessor(processorElement, encryptor);
-            final ProcessorNode procNode = processGroup.getProcessor(dto.getId());
-
-            if (!procNode.getScheduledState().name().equals(dto.getState())) {
-                try {
-                    switch (ScheduledState.valueOf(dto.getState())) {
-                        case DISABLED:
-                            // switch processor do disabled. This means we have to stop it (if it's already stopped, this method does nothing),
-                            // and then we have to disable it.
-                            procNode.getProcessGroup().stopProcessor(procNode);
-                            procNode.getProcessGroup().disableProcessor(procNode);
-                            break;
-                        case RUNNING:
-                            // we want to run now. Make sure processor is not disabled and then start it.
-                            procNode.getProcessGroup().enableProcessor(procNode);
-                            procNode.getProcessGroup().startProcessor(procNode);
-                            break;
-                        case STOPPED:
-                            if (procNode.getScheduledState() == ScheduledState.DISABLED) {
-                                procNode.getProcessGroup().enableProcessor(procNode);
-                            } else if (procNode.getScheduledState() == ScheduledState.RUNNING) {
-                                procNode.getProcessGroup().stopProcessor(procNode);
-                            }
-                            break;
-                    }
-                } catch (final IllegalStateException ise) {
-                    logger.error("Failed to change Scheduled State of {} from {} to {} due to {}", procNode, procNode.getScheduledState().name(), dto.getState(), ise.toString());
-                    logger.error("", ise);
-
-                    // create bulletin for the Processor Node
-                    controller.getBulletinRepository().addBulletin(BulletinFactory.createBulletin(procNode, "Node Reconnection", Severity.ERROR.name(),
-                            "Failed to change Scheduled State of " + procNode + " from " + procNode.getScheduledState().name() + " to " + dto.getState() + " due to " + ise.toString()));
-
-                    // create bulletin at Controller level.
-                    controller.getBulletinRepository().addBulletin(BulletinFactory.createBulletin("Node Reconnection", Severity.ERROR.name(),
-                            "Failed to change Scheduled State of " + procNode + " from " + procNode.getScheduledState().name() + " to " + dto.getState() + " due to " + ise.toString()));
-                }
-            }
-        }
-
-        final List<Element> inputPortList = getChildrenByTagName(processGroupElement, "inputPort");
-        for (final Element portElement : inputPortList) {
-            final PortDTO dto = FlowFromDOMFactory.getPort(portElement);
-            final Port port = processGroup.getInputPort(dto.getId());
-
-            if (!port.getScheduledState().name().equals(dto.getState())) {
-                switch (ScheduledState.valueOf(dto.getState())) {
-                    case DISABLED:
-                        // switch processor do disabled. This means we have to stop it (if it's already stopped, this method does nothing),
-                        // and then we have to disable it.
-                        port.getProcessGroup().stopInputPort(port);
-                        port.getProcessGroup().disableInputPort(port);
-                        break;
-                    case RUNNING:
-                        // we want to run now. Make sure processor is not disabled and then start it.
-                        port.getProcessGroup().enableInputPort(port);
-                        port.getProcessGroup().startInputPort(port);
-                        break;
-                    case STOPPED:
-                        if (port.getScheduledState() == ScheduledState.DISABLED) {
-                            port.getProcessGroup().enableInputPort(port);
-                        } else if (port.getScheduledState() == ScheduledState.RUNNING) {
-                            port.getProcessGroup().stopInputPort(port);
-                        }
-                        break;
-                }
-            }
-        }
-
-        final List<Element> outputPortList = getChildrenByTagName(processGroupElement, "outputPort");
-        for (final Element portElement : outputPortList) {
-            final PortDTO dto = FlowFromDOMFactory.getPort(portElement);
-            final Port port = processGroup.getOutputPort(dto.getId());
-
-            if (!port.getScheduledState().name().equals(dto.getState())) {
-                switch (ScheduledState.valueOf(dto.getState())) {
-                    case DISABLED:
-                        // switch processor do disabled. This means we have to stop it (if it's already stopped, this method does nothing),
-                        // and then we have to disable it.
-                        port.getProcessGroup().stopOutputPort(port);
-                        port.getProcessGroup().disableOutputPort(port);
-                        break;
-                    case RUNNING:
-                        // we want to run now. Make sure processor is not disabled and then start it.
-                        port.getProcessGroup().enableOutputPort(port);
-                        port.getProcessGroup().startOutputPort(port);
-                        break;
-                    case STOPPED:
-                        if (port.getScheduledState() == ScheduledState.DISABLED) {
-                            port.getProcessGroup().enableOutputPort(port);
-                        } else if (port.getScheduledState() == ScheduledState.RUNNING) {
-                            port.getProcessGroup().stopOutputPort(port);
-                        }
-                        break;
-                }
-            }
-        }
-
-        // add labels
-        final List<Element> labelNodeList = getChildrenByTagName(processGroupElement, "label");
-        for (final Element labelElement : labelNodeList) {
-            final LabelDTO labelDTO = FlowFromDOMFactory.getLabel(labelElement);
-            final Label label = controller.createLabel(labelDTO.getId(), labelDTO.getLabel());
-            label.setStyle(labelDTO.getStyle());
-            label.setPosition(new Position(labelDTO.getPosition().getX(), labelDTO.getPosition().getY()));
-            if (labelDTO.getWidth() != null && labelDTO.getHeight() != null) {
-                label.setSize(new Size(labelDTO.getWidth(), labelDTO.getHeight()));
-            }
-
-            processGroup.addLabel(label);
-        }
-
-        // update nested process groups (recursively)
-        final List<Element> nestedProcessGroupNodeList = getChildrenByTagName(processGroupElement, "processGroup");
-        for (final Element nestedProcessGroupElement : nestedProcessGroupNodeList) {
-            updateProcessGroup(controller, processGroup, nestedProcessGroupElement, encryptor);
-        }
-
-        // update connections
-        final List<Element> connectionNodeList = getChildrenByTagName(processGroupElement, "connection");
-        for (final Element connectionElement : connectionNodeList) {
-            final ConnectionDTO dto = FlowFromDOMFactory.getConnection(connectionElement);
-
-            final Connection connection = processGroup.getConnection(dto.getId());
-            connection.setName(dto.getName());
-            connection.setProcessGroup(processGroup);
-
-            if (dto.getLabelIndex() != null) {
-                connection.setLabelIndex(dto.getLabelIndex());
-            }
-            if (dto.getzIndex() != null) {
-                connection.setZIndex(dto.getzIndex());
-            }
-
-            final List<Position> bendPoints = new ArrayList<>();
-            for (final PositionDTO bend : dto.getBends()) {
-                bendPoints.add(new Position(bend.getX(), bend.getY()));
-            }
-            connection.setBendPoints(bendPoints);
-
-            List<FlowFilePrioritizer> newPrioritizers = null;
-            final List<String> prioritizers = dto.getPrioritizers();
-            if (prioritizers != null) {
-                final List<String> newPrioritizersClasses = new ArrayList<>(prioritizers);
-                newPrioritizers = new ArrayList<>();
-                for (final String className : newPrioritizersClasses) {
-                    try {
-                        newPrioritizers.add(controller.createPrioritizer(className));
-                    } catch (final ClassNotFoundException | InstantiationException | IllegalAccessException e) {
-                        throw new IllegalArgumentException("Unable to set prioritizer " + className + ": " + e);
-                    }
-                }
-            }
-
-            if (newPrioritizers != null) {
-                connection.getFlowFileQueue().setPriorities(newPrioritizers);
-            }
-
-            if (dto.getBackPressureObjectThreshold() != null) {
-                connection.getFlowFileQueue().setBackPressureObjectThreshold(dto.getBackPressureObjectThreshold());
-            }
-
-            if (dto.getBackPressureDataSizeThreshold() != null && !dto.getBackPressureDataSizeThreshold().trim().isEmpty()) {
-                connection.getFlowFileQueue().setBackPressureDataSizeThreshold(dto.getBackPressureDataSizeThreshold());
-            }
-
-            if (dto.getFlowFileExpiration() != null) {
-                connection.getFlowFileQueue().setFlowFileExpiration(dto.getFlowFileExpiration());
-            }
-        }
-
-        return processGroup;
-    }
-
-    private Position toPosition(final PositionDTO dto) {
-        return new Position(dto.getX(), dto.getY());
-    }
-
-    private void updateProcessor(final ProcessorNode procNode, final ProcessorDTO processorDTO, final ProcessGroup processGroup, final FlowController controller) throws ProcessorInstantiationException {
-        final ProcessorConfigDTO config = processorDTO.getConfig();
-        procNode.setPosition(toPosition(processorDTO.getPosition()));
-        procNode.setName(processorDTO.getName());
-        procNode.setStyle(processorDTO.getStyle());
-        procNode.setProcessGroup(processGroup);
-        procNode.setComments(config.getComments());
-        procNode.setLossTolerant(config.isLossTolerant());
-        procNode.setPenalizationPeriod(config.getPenaltyDuration());
-        procNode.setYieldPeriod(config.getYieldDuration());
-        procNode.setBulletinLevel(LogLevel.valueOf(config.getBulletinLevel()));
-
-        if (config.getSchedulingStrategy() != null) {
-            procNode.setSchedulingStrategy(SchedulingStrategy.valueOf(config.getSchedulingStrategy()));
-        }
-
-        // must set scheduling strategy before these two
-        procNode.setMaxConcurrentTasks(config.getConcurrentlySchedulableTaskCount());
-        procNode.setScheduldingPeriod(config.getSchedulingPeriod());
-        if (config.getRunDurationMillis() != null) {
-            procNode.setRunDuration(config.getRunDurationMillis(), TimeUnit.MILLISECONDS);
-        }
-
-        procNode.setAnnotationData(config.getAnnotationData());
-
-        if (config.getAutoTerminatedRelationships() != null) {
-            final Set<Relationship> relationships = new HashSet<>();
-            for (final String rel : config.getAutoTerminatedRelationships()) {
-                relationships.add(procNode.getRelationship(rel));
-            }
-            procNode.setAutoTerminatedRelationships(relationships);
-        }
-
-        for (final Map.Entry<String, String> entry : config.getProperties().entrySet()) {
-            if (entry.getValue() == null) {
-                procNode.removeProperty(entry.getKey());
-            } else {
-                procNode.setProperty(entry.getKey(), entry.getValue());
-            }
-        }
-
-        final ScheduledState scheduledState = ScheduledState.valueOf(processorDTO.getState());
-        if (ScheduledState.RUNNING.equals(scheduledState)) {
-            controller.startProcessor(processGroup.getIdentifier(), procNode.getIdentifier());
-        } else if (ScheduledState.DISABLED.equals(scheduledState)) {
-            processGroup.disableProcessor(procNode);
-        }
-    }
-
-    private ProcessGroup addProcessGroup(final FlowController controller, final ProcessGroup parentGroup, final Element processGroupElement, final StringEncryptor encryptor) throws ProcessorInstantiationException {
-        // get the parent group ID
-        final String parentId = (parentGroup == null) ? null : parentGroup.getIdentifier();
-
-        // add the process group
-        final ProcessGroupDTO processGroupDTO = FlowFromDOMFactory.getProcessGroup(parentId, processGroupElement, encryptor);
-        final ProcessGroup processGroup = controller.createProcessGroup(processGroupDTO.getId());
-        processGroup.setComments(processGroupDTO.getComments());
-        processGroup.setPosition(toPosition(processGroupDTO.getPosition()));
-        processGroup.setName(processGroupDTO.getName());
-        processGroup.setParent(parentGroup);
-        if (parentGroup == null) {
-            controller.setRootGroup(processGroup);
-        } else {
-            parentGroup.addProcessGroup(processGroup);
-        }
-
-        // add processors
-        final List<Element> processorNodeList = getChildrenByTagName(processGroupElement, "processor");
-        for (final Element processorElement : processorNodeList) {
-            final ProcessorDTO processorDTO = FlowFromDOMFactory.getProcessor(processorElement, encryptor);
-            final ProcessorNode procNode = controller.createProcessor(processorDTO.getType(), processorDTO.getId(), false);
-            processGroup.addProcessor(procNode);
-            updateProcessor(procNode, processorDTO, processGroup, controller);
-        }
-
-        // add input ports
-        final List<Element> inputPortNodeList = getChildrenByTagName(processGroupElement, "inputPort");
-        for (final Element inputPortElement : inputPortNodeList) {
-            final PortDTO portDTO = FlowFromDOMFactory.getPort(inputPortElement);
-
-            final Port port;
-            if (processGroup.isRootGroup()) {
-                port = controller.createRemoteInputPort(portDTO.getId(), portDTO.getName());
-            } else {
-                port = controller.createLocalInputPort(portDTO.getId(), portDTO.getName());
-            }
-
-            port.setPosition(toPosition(portDTO.getPosition()));
-            port.setComments(portDTO.getComments());
-            port.setProcessGroup(processGroup);
-
-            final Set<String> userControls = portDTO.getUserAccessControl();
-            if (userControls != null && !userControls.isEmpty()) {
-                if (!(port instanceof RootGroupPort)) {
-                    throw new IllegalStateException("Attempting to add User Access Controls to " + port + ", but it is not a RootGroupPort");
-                }
-                ((RootGroupPort) port).setUserAccessControl(userControls);
-            }
-            final Set<String> groupControls = portDTO.getGroupAccessControl();
-            if (groupControls != null && !groupControls.isEmpty()) {
-                if (!(port instanceof RootGroupPort)) {
-                    throw new IllegalStateException("Attempting to add Group Access Controls to " + port + ", but it is not a RootGroupPort");
-                }
-                ((RootGroupPort) port).setGroupAccessControl(groupControls);
-            }
-
-            processGroup.addInputPort(port);
-            if (portDTO.getConcurrentlySchedulableTaskCount() != null) {
-                port.setMaxConcurrentTasks(portDTO.getConcurrentlySchedulableTaskCount());
-            }
-
-            final ScheduledState scheduledState = ScheduledState.valueOf(portDTO.getState());
-            if (ScheduledState.RUNNING.equals(scheduledState)) {
-                controller.startConnectable(port);
-            } else if (ScheduledState.DISABLED.equals(scheduledState)) {
-                processGroup.disableInputPort(port);
-            }
-        }
-
-        // add output ports
-        final List<Element> outputPortNodeList = getChildrenByTagName(processGroupElement, "outputPort");
-        for (final Element outputPortElement : outputPortNodeList) {
-            final PortDTO portDTO = FlowFromDOMFactory.getPort(outputPortElement);
-
-            final Port port;
-            if (processGroup.isRootGroup()) {
-                port = controller.createRemoteOutputPort(portDTO.getId(), portDTO.getName());
-            } else {
-                port = controller.createLocalOutputPort(portDTO.getId(), portDTO.getName());
-            }
-            port.setPosition(toPosition(portDTO.getPosition()));
-            port.setComments(portDTO.getComments());
-            port.setProcessGroup(processGroup);
-
-            final Set<String> userControls = portDTO.getUserAccessControl();
-            if (userControls != null && !userControls.isEmpty()) {
-                if (!(port instanceof RootGroupPort)) {
-                    throw new IllegalStateException("Attempting to add User Access Controls to " + port + ", but it is not a RootGroupPort");
-                }
-                ((RootGroupPort) port).setUserAccessControl(userControls);
-            }
-            final Set<String> groupControls = portDTO.getGroupAccessControl();
-            if (groupControls != null && !groupControls.isEmpty()) {
-                if (!(port instanceof RootGroupPort)) {
-                    throw new IllegalStateException("Attempting to add Group Access Controls to " + port + ", but it is not a RootGroupPort");
-                }
-                ((RootGroupPort) port).setGroupAccessControl(groupControls);
-            }
-
-            processGroup.addOutputPort(port);
-            if (portDTO.getConcurrentlySchedulableTaskCount() != null) {
-                port.setMaxConcurrentTasks(portDTO.getConcurrentlySchedulableTaskCount());
-            }
-
-            final ScheduledState scheduledState = ScheduledState.valueOf(portDTO.getState());
-            if (ScheduledState.RUNNING.equals(scheduledState)) {
-                controller.startConnectable(port);
-            } else if (ScheduledState.DISABLED.equals(scheduledState)) {
-                processGroup.disableOutputPort(port);
-            }
-        }
-
-        // add funnels
-        final List<Element> funnelNodeList = getChildrenByTagName(processGroupElement, "funnel");
-        for (final Element funnelElement : funnelNodeList) {
-            final FunnelDTO funnelDTO = FlowFromDOMFactory.getFunnel(funnelElement);
-            final Funnel funnel = controller.createFunnel(funnelDTO.getId());
-            funnel.setPosition(toPosition(funnelDTO.getPosition()));
-            processGroup.addFunnel(funnel);
-            controller.startConnectable(funnel);
-        }
-
-        // add labels
-        final List<Element> labelNodeList = getChildrenByTagName(processGroupElement, "label");
-        for (final Element labelElement : labelNodeList) {
-            final LabelDTO labelDTO = FlowFromDOMFactory.getLabel(labelElement);
-            final Label label = controller.createLabel(labelDTO.getId(), labelDTO.getLabel());
-            label.setStyle(labelDTO.getStyle());
-
-            label.setPosition(toPosition(labelDTO.getPosition()));
-            label.setSize(new Size(labelDTO.getWidth(), labelDTO.getHeight()));
-            processGroup.addLabel(label);
-        }
-
-        // add nested process groups (recursively)
-        final List<Element> nestedProcessGroupNodeList = getChildrenByTagName(processGroupElement, "processGroup");
-        for (final Element nestedProcessGroupElement : nestedProcessGroupNodeList) {
-            addProcessGroup(controller, processGroup, nestedProcessGroupElement, encryptor);
-        }
-
-        // add remote process group
-        final List<Element> remoteProcessGroupNodeList = getChildrenByTagName(processGroupElement, "remoteProcessGroup");
-        for (final Element remoteProcessGroupElement : remoteProcessGroupNodeList) {
-            final RemoteProcessGroupDTO remoteGroupDto = FlowFromDOMFactory.getRemoteProcessGroup(remoteProcessGroupElement);
-            final RemoteProcessGroup remoteGroup = controller.createRemoteProcessGroup(remoteGroupDto.getId(), remoteGroupDto.getTargetUri());
-            remoteGroup.setComments(remoteGroupDto.getComments());
-            remoteGroup.setPosition(toPosition(remoteGroupDto.getPosition()));
-            final String name = remoteGroupDto.getName();
-            if (name != null && !name.trim().isEmpty()) {
-                remoteGroup.setName(name);
-            }
-            remoteGroup.setProcessGroup(processGroup);
-            remoteGroup.setCommunicationsTimeout(remoteGroupDto.getCommunicationsTimeout());
-
-            if (remoteGroupDto.getYieldDuration() != null) {
-                remoteGroup.setYieldDuration(remoteGroupDto.getYieldDuration());
-            }
-
-            final Set<RemoteProcessGroupPortDescriptor> inputPorts = new HashSet<>();
-            for (final Element portElement : getChildrenByTagName(remoteProcessGroupElement, "inputPort")) {
-                inputPorts.add(FlowFromDOMFactory.getRemoteProcessGroupPort(portElement));
-            }
-            remoteGroup.setInputPorts(inputPorts);
-
-            final Set<RemoteProcessGroupPortDescriptor> outputPorts = new HashSet<>();
-            for (final Element portElement : getChildrenByTagName(remoteProcessGroupElement, "outputPort")) {
-                outputPorts.add(FlowFromDOMFactory.getRemoteProcessGroupPort(portElement));
-            }
-            remoteGroup.setOutputPorts(outputPorts);
-            processGroup.addRemoteProcessGroup(remoteGroup);
-
-            for (final RemoteProcessGroupPortDescriptor remoteGroupPortDTO : outputPorts) {
-                final RemoteGroupPort port = remoteGroup.getOutputPort(remoteGroupPortDTO.getId());
-                if (Boolean.TRUE.equals(remoteGroupPortDTO.isTransmitting())) {
-                    controller.startTransmitting(port);
-                }
-            }
-            for (final RemoteProcessGroupPortDescriptor remoteGroupPortDTO : inputPorts) {
-                final RemoteGroupPort port = remoteGroup.getInputPort(remoteGroupPortDTO.getId());
-                if (Boolean.TRUE.equals(remoteGroupPortDTO.isTransmitting())) {
-                    controller.startTransmitting(port);
-                }
-            }
-        }
-
-        // add connections
-        final List<Element> connectionNodeList = getChildrenByTagName(processGroupElement, "connection");
-        for (final Element connectionElement : connectionNodeList) {
-            final ConnectionDTO dto = FlowFromDOMFactory.getConnection(connectionElement);
-
-            final Connectable source;
-            final ConnectableDTO sourceDto = dto.getSource();
-            if (ConnectableType.REMOTE_OUTPUT_PORT.name().equals(sourceDto.getType())) {
-                final RemoteProcessGroup remoteGroup = processGroup.getRemoteProcessGroup(sourceDto.getGroupId());
-                source = remoteGroup.getOutputPort(sourceDto.getId());
-            } else {
-                final ProcessGroup sourceGroup = controller.getGroup(sourceDto.getGroupId());
-                if (sourceGroup == null) {
-                    throw new RuntimeException("Found Invalid ProcessGroup ID for Source: " + dto.getSource().getGroupId());
-                }
-
-                source = sourceGroup.getConnectable(sourceDto.getId());
-            }
-            if (source == null) {
-                throw new RuntimeException("Found Invalid Connectable ID for Source: " + dto.getSource().getId());
-            }
-
-            final Connectable destination;
-            final ConnectableDTO destinationDto = dto.getDestination();
-            if (ConnectableType.REMOTE_INPUT_PORT.name().equals(destinationDto.getType())) {
-                final RemoteProcessGroup remoteGroup = processGroup.getRemoteProcessGroup(destinationDto.getGroupId());
-                destination = remoteGroup.getInputPort(destinationDto.getId());
-            } else {
-                final ProcessGroup destinationGroup = controller.getGroup(destinationDto.getGroupId());
-                if (destinationGroup == null) {
-                    throw new RuntimeException("Found Invalid ProcessGroup ID for Destination: " + dto.getDestination().getGroupId());
-                }
-
-                destination = destinationGroup.getConnectable(destinationDto.getId());
-            }
-            if (destination == null) {
-                throw new RuntimeException("Found Invalid Connectable ID for Destination: " + dto.getDestination().getId());
-            }
-
-            final Connection connection = controller.createConnection(dto.getId(), dto.getName(), source, destination, dto.getSelectedRelationships());
-            connection.setProcessGroup(processGroup);
-
-            final List<Position> bendPoints = new ArrayList<>();
-            for (final PositionDTO bend : dto.getBends()) {
-                bendPoints.add(new Position(bend.getX(), bend.getY()));
-            }
-            connection.setBendPoints(bendPoints);
-
-            final Long zIndex = dto.getzIndex();
-            if (zIndex != null) {
-                connection.setZIndex(zIndex);
-            }
-
-            if (dto.getLabelIndex() != null) {
-                connection.setLabelIndex(dto.getLabelIndex());
-            }
-
-            List<FlowFilePrioritizer> newPrioritizers = null;
-            final List<String> prioritizers = dto.getPrioritizers();
-            if (prioritizers != null) {
-                final List<String> newPrioritizersClasses = new ArrayList<>(prioritizers);
-                newPrioritizers = new ArrayList<>();
-                for (final String className : newPrioritizersClasses) {
-                    try {
-                        newPrioritizers.add(controller.createPrioritizer(className));
-                    } catch (final ClassNotFoundException | InstantiationException | IllegalAccessException e) {
-                        throw new IllegalArgumentException("Unable to set prioritizer " + className + ": " + e);
-                    }
-                }
-            }
-            if (newPrioritizers != null) {
-                connection.getFlowFileQueue().setPriorities(newPrioritizers);
-            }
-
-            if (dto.getBackPressureObjectThreshold() != null) {
-                connection.getFlowFileQueue().setBackPressureObjectThreshold(dto.getBackPressureObjectThreshold());
-            }
-            if (dto.getBackPressureDataSizeThreshold() != null) {
-                connection.getFlowFileQueue().setBackPressureDataSizeThreshold(dto.getBackPressureDataSizeThreshold());
-            }
-            if (dto.getFlowFileExpiration() != null) {
-                connection.getFlowFileQueue().setFlowFileExpiration(dto.getFlowFileExpiration());
-            }
-
-            processGroup.addConnection(connection);
-        }
-
-        return processGroup;
-    }
-
-    /**
-     * Returns true if the given controller can inherit the proposed flow
-     * without orphaning flow files.
-     *
-     * @param existingFlow
-     * @param controller the running controller
-     * @param proposedFlow the flow to inherit
-     *
-     * @return null if the controller can inherit the specified flow, an
-     * explanation of why it cannot be inherited otherwise
-     *
-     * @throws FingerprintException if flow fingerprints could not be generated
-     */
-    public String checkFlowInheritability(final DataFlow existingFlow, final DataFlow proposedFlow, final FlowController controller) throws FingerprintException {
-        if (existingFlow == null) {
-            return null;  // no existing flow, so equivalent to proposed flow
-        }
-
-        return checkFlowInheritability(existingFlow.getFlow(), proposedFlow.getFlow(), controller);
-    }
-
-    private String checkFlowInheritability(final byte[] existingFlow, final byte[] proposedFlow, final FlowController controller) {
-        if (existingFlow == null) {
-            return null; // no existing flow, so equivalent to proposed flow
-        }
-
-        // check if the Flow is inheritable
-        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor);
-        final String existingFlowFingerprintBeforeHash = fingerprintFactory.createFingerprint(existingFlow, controller);
-        if (existingFlowFingerprintBeforeHash.trim().isEmpty()) {
-            return null;  // no existing flow, so equivalent to proposed flow
-        }
-
-        if (proposedFlow == null || proposedFlow.length == 0) {
-            return "Proposed Flow was empty but Current Flow is not";  // existing flow is not empty and proposed flow is empty (we could orphan flowfiles)
-        }
-
-        final String proposedFlowFingerprintBeforeHash = fingerprintFactory.createFingerprint(proposedFlow, controller);
-        if (proposedFlowFingerprintBeforeHash.trim().isEmpty()) {
-            return "Proposed Flow was empty but Current Flow is not";  // existing flow is not empty and proposed flow is empty (we could orphan flowfiles)
-        }
-
-        final boolean inheritable = existingFlowFingerprintBeforeHash.equals(proposedFlowFingerprintBeforeHash);
-        if (!inheritable) {
-            return findFirstDiscrepancy(existingFlowFingerprintBeforeHash, proposedFlowFingerprintBeforeHash, "Flows");
-        }
-
-        return null;
-    }
-
-    /**
-     * Returns true if the given controller can inherit the proposed flow
-     * without orphaning flow files.
-     *
-     * @param existingFlow
-     * @param proposedFlow the flow to inherit
-     *
-     * @return null if the controller can inherit the specified flow, an
-     * explanation of why it cannot be inherited otherwise
-     *
-     * @throws FingerprintException if flow fingerprints could not be generated
-     */
-    public String checkTemplateInheritability(final DataFlow existingFlow, final DataFlow proposedFlow) throws FingerprintException {
-        if (existingFlow == null) {
-            return null;  // no existing flow, so equivalent to proposed flow
-        }
-
-        // check if the Flow is inheritable
-        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor);
-        // check if the Templates are inheritable
-        final byte[] existingTemplateBytes = existingFlow.getTemplates();
-        if (existingTemplateBytes == null || existingTemplateBytes.length == 0) {
-            return null;
-        }
-
-        final List<Template> existingTemplates = TemplateManager.parseBytes(existingTemplateBytes);
-        final String existingTemplateFingerprint = fingerprintFactory.createFingerprint(existingTemplates);
-        if (existingTemplateFingerprint.trim().isEmpty()) {
-            return null;
-        }
-
-        final byte[] proposedTemplateBytes = proposedFlow.getTemplates();
-        if (proposedTemplateBytes == null || proposedTemplateBytes.length == 0) {
-            return "Proposed Flow does not contain any Templates but Current Flow does";
-        }
-
-        final List<Template> proposedTemplates = TemplateManager.parseBytes(proposedTemplateBytes);
-        final String proposedTemplateFingerprint = fingerprintFactory.createFingerprint(proposedTemplates);
-        if (proposedTemplateFingerprint.trim().isEmpty()) {
-            return "Proposed Flow does not contain any Templates but Current Flow does";
-        }
-
-        try {
-            final String existingTemplateMd5 = fingerprintFactory.md5Hash(existingTemplateFingerprint);
-            final String proposedTemplateMd5 = fingerprintFactory.md5Hash(proposedTemplateFingerprint);
-
-            if (!existingTemplateMd5.equals(proposedTemplateMd5)) {
-                return findFirstDiscrepancy(existingTemplateFingerprint, proposedTemplateFingerprint, "Templates");
-            }
-        } catch (final NoSuchAlgorithmException e) {
-            throw new FingerprintException(e);
-        }
-
-        return null;
-    }
-
-    private String findFirstDiscrepancy(final String existing, final String proposed, final String comparisonDescription) {
-        final int shortestFileLength = Math.min(existing.length(), proposed.length());
-        for (int i = 0; i < shortestFileLength; i++) {
-            if (existing.charAt(i) != proposed.charAt(i)) {
-                final String formattedExistingDelta = formatFlowDiscrepancy(existing, i, 100);
-                final String formattedProposedDelta = formatFlowDiscrepancy(proposed, i, 100);
-                return String.format("Found difference in %s:\nLocal Fingerprint:   %s\nCluster Fingerprint: %s", comparisonDescription, formattedExistingDelta, formattedProposedDelta);
-            }
-        }
-
-        // existing must startWith proposed or proposed must startWith existing
-        if (existing.length() > proposed.length()) {
-            final String formattedExistingDelta = existing.substring(proposed.length(), Math.min(existing.length(), proposed.length() + 200));
-            return String.format("Found difference in %s:\nLocal Fingerprint contains additional configuration from Cluster Fingerprint: %s", comparisonDescription, formattedExistingDelta);
-        } else if (proposed.length() > existing.length()) {
-            final String formattedProposedDelta = proposed.substring(existing.length(), Math.min(proposed.length(), existing.length() + 200));
-            return String.format("Found difference in %s:\nCluster Fingerprint contains additional configuration from Local Fingerprint: %s", comparisonDescription, formattedProposedDelta);
-        }
-
-        return "Unable to find any discrepancies between fingerprints. Please contact the NiFi support team";
-    }
-
-    private byte[] toBytes(final FlowController flowController) throws FlowSerializationException {
-        final ByteArrayOutputStream result = new ByteArrayOutputStream();
-        final StandardFlowSerializer flowSerializer = new StandardFlowSerializer(encryptor);
-        flowController.serialize(flowSerializer, result);
-        return result.toByteArray();
-    }
-
-    private static String getString(final Element element, final String childElementName) {
-        final List<Element> nodeList = getChildrenByTagName(element, childElementName);
-        if (nodeList == null || nodeList.isEmpty()) {
-            return "";
-        }
-        final Element childElement = nodeList.get(0);
-        return childElement.getTextContent();
-    }
-
-    private static int getInt(final Element element, final String childElementName) {
-        return Integer.parseInt(getString(element, childElementName));
-    }
-
-    private static Integer getInteger(final Element element, final String childElementName) {
-        final String value = getString(element, childElementName);
-        return (value == null || value.trim().equals("") ? null : Integer.parseInt(value));
-    }
-
-    private static List<Element> getChildrenByTagName(final Element element, final String tagName) {
-        final List<Element> matches = new ArrayList<>();
-        final NodeList nodeList = element.getChildNodes();
-        for (int i = 0; i < nodeList.getLength(); i++) {
-            final Node node = nodeList.item(i);
-            if (!(node instanceof Element)) {
-                continue;
-            }
-
-            final Element child = (Element) nodeList.item(i);
-            if (child.getNodeName().equals(tagName)) {
-                matches.add(child);
-            }
-        }
-
-        return matches;
-    }
-
-    private String formatFlowDiscrepancy(final String flowFingerprint, final int deltaIndex, final int deltaPad) {
-        return flowFingerprint.substring(Math.max(0, deltaIndex - deltaPad), Math.min(flowFingerprint.length(), deltaIndex + deltaPad));
-    }
-}


[36/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/test/java/org/apache/nifi/util/file/monitor/TestCompoundUpdateMonitor.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/test/java/org/apache/nifi/util/file/monitor/TestCompoundUpdateMonitor.java b/commons/nifi-utils/src/test/java/org/apache/nifi/util/file/monitor/TestCompoundUpdateMonitor.java
deleted file mode 100644
index f576e94..0000000
--- a/commons/nifi-utils/src/test/java/org/apache/nifi/util/file/monitor/TestCompoundUpdateMonitor.java
+++ /dev/null
@@ -1,71 +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.file.monitor;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotSame;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.file.Path;
-import java.util.UUID;
-
-
-import org.junit.Test;
-
-public class TestCompoundUpdateMonitor {
-
-    @Test
-    public void test() throws IOException {
-        final UpdateMonitor lastModified = new LastModifiedMonitor();
-        final MD5SumMonitor md5 = new MD5SumMonitor();
-        final CompoundUpdateMonitor compound = new CompoundUpdateMonitor(lastModified, md5);
-
-        final File file = new File("target/" + UUID.randomUUID().toString());
-        if (file.exists()) {
-            assertTrue(file.delete());
-        }
-        assertTrue(file.createNewFile());
-
-        final Path path = file.toPath();
-
-        final Object curState = compound.getCurrentState(path);
-        final Object state2 = compound.getCurrentState(path);
-
-        assertEquals(curState, state2);
-        file.setLastModified(System.currentTimeMillis() + 1000L);
-        final Object state3 = compound.getCurrentState(path);
-        assertEquals(state2, state3);
-
-        final Object state4 = compound.getCurrentState(path);
-        assertEquals(state3, state4);
-
-        final long lastModifiedDate = file.lastModified();
-        try (final OutputStream out = new FileOutputStream(file)) {
-            out.write("Hello".getBytes("UTF-8"));
-        }
-
-        file.setLastModified(lastModifiedDate);
-
-        final Object state5 = compound.getCurrentState(path);
-        assertNotSame(state4, state5);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/test/java/org/apache/nifi/util/file/monitor/TestSynchronousFileWatcher.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/test/java/org/apache/nifi/util/file/monitor/TestSynchronousFileWatcher.java b/commons/nifi-utils/src/test/java/org/apache/nifi/util/file/monitor/TestSynchronousFileWatcher.java
deleted file mode 100644
index 7125581..0000000
--- a/commons/nifi-utils/src/test/java/org/apache/nifi/util/file/monitor/TestSynchronousFileWatcher.java
+++ /dev/null
@@ -1,61 +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.file.monitor;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.ByteArrayInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.nio.file.StandardCopyOption;
-
-import org.junit.Test;
-
-
-public class TestSynchronousFileWatcher {
-
-    @Test
-    public void testIt() throws UnsupportedEncodingException, IOException, InterruptedException {
-        final Path path = Paths.get("target/1.txt");
-        Files.copy(new ByteArrayInputStream("Hello, World!".getBytes("UTF-8")), path, StandardCopyOption.REPLACE_EXISTING);
-        final UpdateMonitor monitor = new MD5SumMonitor();
-
-        final SynchronousFileWatcher watcher = new SynchronousFileWatcher(path, monitor, 10L);
-        assertFalse(watcher.checkAndReset());
-        Thread.sleep(30L);
-        assertFalse(watcher.checkAndReset());
-
-        final FileOutputStream fos = new FileOutputStream(path.toFile());
-        try {
-            fos.write("Good-bye, World!".getBytes("UTF-8"));
-            fos.getFD().sync();
-        } finally {
-            fos.close();
-        }
-
-        assertTrue(watcher.checkAndReset());
-        assertFalse(watcher.checkAndReset());
-
-        Thread.sleep(30L);
-        assertFalse(watcher.checkAndReset());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestRingBuffer.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestRingBuffer.java b/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestRingBuffer.java
deleted file mode 100644
index fafffdd..0000000
--- a/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestRingBuffer.java
+++ /dev/null
@@ -1,182 +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.timebuffer;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.nifi.util.RingBuffer;
-import org.apache.nifi.util.RingBuffer.ForEachEvaluator;
-import org.apache.nifi.util.RingBuffer.IterationDirection;
-
-import org.junit.Test;
-
-/**
- *
- */
-public class TestRingBuffer {
-
-    @Test
-    public void testAsList() {
-        final RingBuffer<Integer> ringBuffer = new RingBuffer<>(10);
-
-        final List<Integer> emptyList = ringBuffer.asList();
-        assertTrue(emptyList.isEmpty());
-
-        for (int i = 0; i < 3; i++) {
-            ringBuffer.add(i);
-        }
-
-        List<Integer> list = ringBuffer.asList();
-        assertEquals(3, list.size());
-        for (int i = 0; i < 3; i++) {
-            assertEquals(Integer.valueOf(i), list.get(i));
-        }
-
-        for (int i = 3; i < 10; i++) {
-            ringBuffer.add(i);
-        }
-
-        list = ringBuffer.asList();
-        assertEquals(10, list.size());
-        for (int i = 0; i < 10; i++) {
-            assertEquals(Integer.valueOf(i), list.get(i));
-        }
-    }
-
-    @Test
-    public void testIterateForwards() {
-        final RingBuffer<Integer> ringBuffer = new RingBuffer<>(10);
-
-        final int[] values = new int[]{3, 5, 20, 7};
-        for (final int v : values) {
-            ringBuffer.add(v);
-        }
-
-        final AtomicInteger countHolder = new AtomicInteger(0);
-        ringBuffer.forEach(new ForEachEvaluator<Integer>() {
-            int counter = 0;
-
-            @Override
-            public boolean evaluate(final Integer value) {
-                final int expected = values[counter++];
-                countHolder.incrementAndGet();
-                assertEquals(expected, value.intValue());
-                return true;
-            }
-
-        }, IterationDirection.FORWARD);
-
-        assertEquals(4, countHolder.get());
-    }
-
-    @Test
-    public void testIterateForwardsAfterFull() {
-        final RingBuffer<Integer> ringBuffer = new RingBuffer<>(10);
-
-        for (int i = 0; i < 12; i++) {
-            ringBuffer.add(i);
-        }
-
-        final int[] values = new int[]{3, 5, 20, 7};
-        for (final int v : values) {
-            ringBuffer.add(v);
-        }
-
-        ringBuffer.forEach(new ForEachEvaluator<Integer>() {
-            int counter = 0;
-
-            @Override
-            public boolean evaluate(final Integer value) {
-                if (counter < 6) {
-                    assertEquals(counter + 6, value.intValue());
-                } else {
-                    final int expected = values[counter - 6];
-                    assertEquals(expected, value.intValue());
-                }
-
-                counter++;
-                return true;
-            }
-
-        }, IterationDirection.FORWARD);
-    }
-
-    @Test
-    public void testIterateBackwards() {
-        final RingBuffer<Integer> ringBuffer = new RingBuffer<>(10);
-
-        final int[] values = new int[]{3, 5, 20, 7};
-        for (final int v : values) {
-            ringBuffer.add(v);
-        }
-
-        final AtomicInteger countHolder = new AtomicInteger(0);
-        ringBuffer.forEach(new ForEachEvaluator<Integer>() {
-            int counter = 0;
-
-            @Override
-            public boolean evaluate(final Integer value) {
-                final int index = values.length - 1 - counter;
-                final int expected = values[index];
-                countHolder.incrementAndGet();
-
-                assertEquals(expected, value.intValue());
-                counter++;
-                return true;
-            }
-
-        }, IterationDirection.BACKWARD);
-
-        assertEquals(4, countHolder.get());
-    }
-
-    @Test
-    public void testIterateBackwardsAfterFull() {
-        final RingBuffer<Integer> ringBuffer = new RingBuffer<>(10);
-
-        for (int i = 0; i < 12; i++) {
-            ringBuffer.add(i);
-        }
-
-        final int[] values = new int[]{3, 5, 20, 7};
-        for (final int v : values) {
-            ringBuffer.add(v);
-        }
-
-        ringBuffer.forEach(new ForEachEvaluator<Integer>() {
-            int counter = 0;
-
-            @Override
-            public boolean evaluate(final Integer value) {
-                if (counter < values.length) {
-                    final int index = values.length - 1 - counter;
-                    final int expected = values[index];
-
-                    assertEquals(expected, value.intValue());
-                    counter++;
-                }
-
-                return true;
-            }
-
-        }, IterationDirection.BACKWARD);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestTimedBuffer.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestTimedBuffer.java b/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestTimedBuffer.java
deleted file mode 100644
index 39ca330..0000000
--- a/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestTimedBuffer.java
+++ /dev/null
@@ -1,106 +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.timebuffer;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
-import java.util.concurrent.TimeUnit;
-
-import org.junit.Test;
-
-public class TestTimedBuffer {
-
-    @Test
-    public void testAgesOff() throws InterruptedException {
-        final LongEntityAccess access = new LongEntityAccess();
-        final TimedBuffer<TimestampedLong> buffer = new TimedBuffer<>(TimeUnit.SECONDS, 2, access);
-
-        buffer.add(new TimestampedLong(1000000L));
-        TimestampedLong aggregate = buffer.getAggregateValue(System.currentTimeMillis() - 30000L);
-        assertEquals(1000000L, aggregate.getValue().longValue());
-        Thread.sleep(1000L);
-        aggregate = buffer.getAggregateValue(System.currentTimeMillis() - 30000L);
-        assertEquals(1000000L, aggregate.getValue().longValue());
-        Thread.sleep(1500L);
-        aggregate = buffer.getAggregateValue(System.currentTimeMillis() - 30000L);
-        assertNull(aggregate);
-    }
-
-    @Test
-    public void testAggregation() throws InterruptedException {
-        final LongEntityAccess access = new LongEntityAccess();
-        final TimedBuffer<TimestampedLong> buffer = new TimedBuffer<>(TimeUnit.SECONDS, 2, access);
-
-        buffer.add(new TimestampedLong(1000000L));
-        buffer.add(new TimestampedLong(1000000L));
-        buffer.add(new TimestampedLong(25000L));
-
-        TimestampedLong aggregate = buffer.getAggregateValue(System.currentTimeMillis() - 30000L);
-        assertEquals(2025000L, aggregate.getValue().longValue());
-        Thread.sleep(1000L);
-        aggregate = buffer.getAggregateValue(System.currentTimeMillis() - 30000L);
-        assertEquals(2025000L, aggregate.getValue().longValue());
-        Thread.sleep(1500L);
-        aggregate = buffer.getAggregateValue(System.currentTimeMillis() - 30000L);
-        assertNull(aggregate);
-    }
-
-    private static class TimestampedLong {
-
-        private final Long value;
-        private final long timestamp = System.currentTimeMillis();
-
-        public TimestampedLong(final Long value) {
-            this.value = value;
-        }
-
-        public Long getValue() {
-            return value;
-        }
-
-        public long getTimestamp() {
-            return timestamp;
-        }
-    }
-
-    private static class LongEntityAccess implements EntityAccess<TimestampedLong> {
-
-        @Override
-        public TimestampedLong aggregate(TimestampedLong oldValue, TimestampedLong toAdd) {
-            if (oldValue == null && toAdd == null) {
-                return new TimestampedLong(0L);
-            } else if (oldValue == null) {
-                return toAdd;
-            } else if (toAdd == null) {
-                return oldValue;
-            }
-
-            return new TimestampedLong(oldValue.getValue().longValue() + toAdd.getValue().longValue());
-        }
-
-        @Override
-        public TimestampedLong createNew() {
-            return new TimestampedLong(0L);
-        }
-
-        @Override
-        public long getTimestamp(TimestampedLong entity) {
-            return entity == null ? 0L : entity.getTimestamp();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-utils/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/test/resources/logback-test.xml b/commons/nifi-utils/src/test/resources/logback-test.xml
deleted file mode 100644
index 8651d47..0000000
--- a/commons/nifi-utils/src/test/resources/logback-test.xml
+++ /dev/null
@@ -1,32 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
-<configuration scan="true" scanPeriod="30 seconds">
-    <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender">
-        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
-            <pattern>%-4r [%t] %-5p %c - %m%n</pattern>
-        </encoder>
-    </appender>
-    
-    <!-- valid logging levels: TRACE, DEBUG, INFO, WARN, ERROR -->
-    <logger name="org.apache.nifi" level="DEBUG"/>
-    
-    <root level="INFO">
-        <appender-ref ref="CONSOLE"/>
-    </root>
-    
-</configuration>
-

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-web-utils/pom.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-web-utils/pom.xml b/commons/nifi-web-utils/pom.xml
deleted file mode 100644
index 63d6e57..0000000
--- a/commons/nifi-web-utils/pom.xml
+++ /dev/null
@@ -1,56 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-commons-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-
-    <artifactId>nifi-web-utils</artifactId>
-    <version>0.0.1-SNAPSHOT</version>
-    <name>NiFi Web Utils</name>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-security-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-codec</groupId>
-            <artifactId>commons-codec</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-lang3</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.sun.jersey</groupId>
-            <artifactId>jersey-client</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.sun.jersey</groupId>
-            <artifactId>jersey-json</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>javax.servlet</groupId>
-            <artifactId>javax.servlet-api</artifactId>
-            <scope>provided</scope>
-        </dependency>
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/ClientUtils.java
----------------------------------------------------------------------
diff --git a/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/ClientUtils.java b/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/ClientUtils.java
deleted file mode 100644
index 8c0b1f4..0000000
--- a/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/ClientUtils.java
+++ /dev/null
@@ -1,132 +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.util;
-
-import com.sun.jersey.api.client.Client;
-import com.sun.jersey.api.client.ClientHandlerException;
-import com.sun.jersey.api.client.ClientResponse;
-import com.sun.jersey.api.client.UniformInterfaceException;
-import com.sun.jersey.api.client.WebResource;
-import com.sun.jersey.core.util.MultivaluedMapImpl;
-import java.net.URI;
-import java.util.Map;
-import javax.ws.rs.core.MediaType;
-
-/**
- *
- */
-public class ClientUtils {
-
-    private final Client client;
-
-    public ClientUtils(Client client) {
-        this.client = client;
-    }
-
-    /**
-     * Gets the content at the specified URI.
-     *
-     * @param uri
-     * @return
-     * @throws ClientHandlerException
-     * @throws UniformInterfaceException
-     */
-    public ClientResponse get(final URI uri) throws ClientHandlerException, UniformInterfaceException {
-        return get(uri, null);
-    }
-
-    /**
-     * Gets the content at the specified URI using the given query parameters.
-     *
-     * @param uri
-     * @param queryParams
-     * @return
-     * @throws ClientHandlerException
-     * @throws UniformInterfaceException
-     */
-    public ClientResponse get(final URI uri, final Map<String, String> queryParams) throws ClientHandlerException, UniformInterfaceException {
-        // perform the request
-        WebResource webResource = client.resource(uri);
-        if (queryParams != null) {
-            for (final Map.Entry<String, String> queryEntry : queryParams.entrySet()) {
-                webResource = webResource.queryParam(queryEntry.getKey(), queryEntry.getValue());
-            }
-        }
-
-        return webResource.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    }
-
-    /**
-     * Performs a POST using the specified url and entity body.
-     *
-     * @param uri
-     * @param entity
-     * @return
-     */
-    public ClientResponse post(URI uri, Object entity) throws ClientHandlerException, UniformInterfaceException {
-        // get the resource
-        WebResource.Builder resourceBuilder = client.resource(uri).accept(MediaType.APPLICATION_JSON).type(MediaType.APPLICATION_JSON);
-
-        // include the request entity
-        if (entity != null) {
-            resourceBuilder = resourceBuilder.entity(entity);
-        }
-
-        // perform the request
-        return resourceBuilder.post(ClientResponse.class);
-    }
-
-    /**
-     * Performs a POST using the specified url and form data.
-     *
-     * @param uri
-     * @param formData
-     * @return
-     */
-    public ClientResponse post(URI uri, Map<String, String> formData) throws ClientHandlerException, UniformInterfaceException {
-        // convert the form data
-        MultivaluedMapImpl entity = new MultivaluedMapImpl();
-        for (String key : formData.keySet()) {
-            entity.add(key, formData.get(key));
-        }
-
-        // get the resource
-        WebResource.Builder resourceBuilder = client.resource(uri).accept(MediaType.APPLICATION_JSON).type(MediaType.APPLICATION_FORM_URLENCODED);
-
-        // add the form data if necessary
-        if (!entity.isEmpty()) {
-            resourceBuilder = resourceBuilder.entity(entity);
-        }
-
-        // perform the request
-        return resourceBuilder.post(ClientResponse.class);
-    }
-
-    /**
-     * Performs a HEAD request to the specified URI.
-     *
-     * @param uri
-     * @return
-     * @throws ClientHandlerException
-     * @throws UniformInterfaceException
-     */
-    public ClientResponse head(final URI uri) throws ClientHandlerException, UniformInterfaceException {
-        // perform the request
-        WebResource webResource = client.resource(uri);
-        return webResource.head();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/ObjectMapperResolver.java
----------------------------------------------------------------------
diff --git a/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/ObjectMapperResolver.java b/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/ObjectMapperResolver.java
deleted file mode 100644
index 4e7f5b6..0000000
--- a/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/ObjectMapperResolver.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.web.util;
-
-import javax.ws.rs.ext.ContextResolver;
-import javax.ws.rs.ext.Provider;
-import org.codehaus.jackson.map.AnnotationIntrospector;
-import org.codehaus.jackson.map.DeserializationConfig;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.map.SerializationConfig;
-import org.codehaus.jackson.map.annotate.JsonSerialize.Inclusion;
-import org.codehaus.jackson.xc.JaxbAnnotationIntrospector;
-
-@Provider
-public class ObjectMapperResolver implements ContextResolver<ObjectMapper> {
-
-    private final ObjectMapper mapper;
-
-    public ObjectMapperResolver() throws Exception {
-        mapper = new ObjectMapper();
-
-        final AnnotationIntrospector jaxbIntrospector = new JaxbAnnotationIntrospector();
-        final SerializationConfig serializationConfig = mapper.getSerializationConfig();
-        final DeserializationConfig deserializationConfig = mapper.getDeserializationConfig();
-
-        mapper.setSerializationConfig(serializationConfig.withSerializationInclusion(Inclusion.NON_NULL).withAnnotationIntrospector(jaxbIntrospector));
-        mapper.setDeserializationConfig(deserializationConfig.withAnnotationIntrospector(jaxbIntrospector));
-    }
-
-    @Override
-    public ObjectMapper getContext(Class<?> objectType) {
-        return mapper;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/WebUtils.java
----------------------------------------------------------------------
diff --git a/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/WebUtils.java b/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/WebUtils.java
deleted file mode 100644
index 587b3d8..0000000
--- a/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/WebUtils.java
+++ /dev/null
@@ -1,198 +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.util;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.Serializable;
-import java.security.cert.Certificate;
-import java.security.cert.CertificateParsingException;
-import java.security.cert.X509Certificate;
-import java.util.List;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import javax.net.ssl.HostnameVerifier;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLPeerUnverifiedException;
-import javax.net.ssl.SSLSession;
-
-import org.apache.nifi.security.util.CertificateUtils;
-
-import org.apache.commons.codec.DecoderException;
-import org.apache.commons.codec.binary.Hex;
-import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.sun.jersey.api.client.Client;
-import com.sun.jersey.api.client.config.ClientConfig;
-import com.sun.jersey.api.client.config.DefaultClientConfig;
-import com.sun.jersey.api.json.JSONConfiguration;
-import com.sun.jersey.client.urlconnection.HTTPSProperties;
-
-/**
- * Common utilities related to web development.
- *
- * @author unattributed
- */
-public final class WebUtils {
-
-    private static Logger logger = LoggerFactory.getLogger(WebUtils.class);
-
-    final static ReadWriteLock lock = new ReentrantReadWriteLock();
-
-    private WebUtils() {
-    }
-
-    /**
-     * Creates a client for non-secure requests. The client will be created
-     * using the given configuration. Additionally, the client will be
-     * automatically configured for JSON serialization/deserialization.
-     *
-     * @param config client configuration
-     *
-     * @return a Client instance
-     */
-    public static Client createClient(final ClientConfig config) {
-        return createClientHelper(config, null);
-    }
-
-    /**
-     * Creates a client for secure requests. The client will be created using
-     * the given configuration and security context. Additionally, the client
-     * will be automatically configured for JSON serialization/deserialization.
-     *
-     * @param config client configuration
-     * @param ctx security context
-     *
-     * @return a Client instance
-     */
-    public static Client createClient(final ClientConfig config, final SSLContext ctx) {
-        return createClientHelper(config, ctx);
-    }
-
-    /**
-     * A helper method for creating clients. The client will be created using
-     * the given configuration and security context. Additionally, the client
-     * will be automatically configured for JSON serialization/deserialization.
-     *
-     * @param config client configuration
-     * @param ctx security context, which may be null for non-secure client
-     * creation
-     *
-     * @return a Client instance
-     */
-    private static Client createClientHelper(final ClientConfig config, final SSLContext ctx) {
-
-        final ClientConfig finalConfig = (config == null) ? new DefaultClientConfig() : config;
-
-        if (ctx != null && StringUtils.isBlank((String) finalConfig.getProperty(HTTPSProperties.PROPERTY_HTTPS_PROPERTIES))) {
-
-            // custom hostname verifier that checks subject alternative names against the hostname of the URI
-            final HostnameVerifier hostnameVerifier = new HostnameVerifier() {
-                @Override
-                public boolean verify(final String hostname, final SSLSession ssls) {
-
-                    try {
-                        for (final Certificate peerCertificate : ssls.getPeerCertificates()) {
-                            if (peerCertificate instanceof X509Certificate) {
-                                final X509Certificate x509Cert = (X509Certificate) peerCertificate;
-                                final List<String> subjectAltNames = CertificateUtils.getSubjectAlternativeNames(x509Cert);
-                                if (subjectAltNames.contains(hostname.toLowerCase())) {
-                                    return true;
-                                }
-                            }
-                        }
-                    } catch (final SSLPeerUnverifiedException | CertificateParsingException ex) {
-                        logger.warn("Hostname Verification encountered exception verifying hostname due to: " + ex, ex);
-                    }
-
-                    return false;
-                }
-            };
-
-            finalConfig.getProperties().put(HTTPSProperties.PROPERTY_HTTPS_PROPERTIES, new HTTPSProperties(hostnameVerifier, ctx));
-        }
-
-        finalConfig.getFeatures().put(JSONConfiguration.FEATURE_POJO_MAPPING, Boolean.TRUE);
-        finalConfig.getClasses().add(ObjectMapperResolver.class);
-
-        // web client for restful request
-        return Client.create(finalConfig);
-
-    }
-
-    /**
-     * Serializes the given object to hexadecimal. Serialization uses Java's
-     * native serialization mechanism, the ObjectOutputStream.
-     *
-     * @param obj an object
-     * @return the serialized object as hex
-     */
-    public static String serializeObjectToHex(final Serializable obj) {
-
-        final ByteArrayOutputStream serializedObj = new ByteArrayOutputStream();
-
-        // IOException can never be thrown because we are serializing to an in memory byte array
-        try {
-            final ObjectOutputStream oos = new ObjectOutputStream(serializedObj);
-            oos.writeObject(obj);
-            oos.close();
-        } catch (final IOException ioe) {
-            throw new RuntimeException(ioe);
-        }
-
-        logger.debug(String.format("Serialized object '%s' size: %d", obj, serializedObj.size()));
-
-        // hex encode the binary
-        return new String(Hex.encodeHex(serializedObj.toByteArray(), /* tolowercase */ true));
-    }
-
-    /**
-     * Deserializes a Java serialized, hex-encoded string into a Java object.
-     * This method is the inverse of the serializeObjectToHex method in this
-     * class.
-     *
-     * @param hexEncodedObject a string
-     * @return the object
-     * @throws ClassNotFoundException if the class could not be found
-     */
-    public static Serializable deserializeHexToObject(final String hexEncodedObject) throws ClassNotFoundException {
-
-        // decode the hex encoded object
-        byte[] serializedObj;
-        try {
-            serializedObj = Hex.decodeHex(hexEncodedObject.toCharArray());
-        } catch (final DecoderException de) {
-            throw new IllegalArgumentException(de);
-        }
-
-        // IOException can never be thrown because we are deserializing from an in memory byte array
-        try {
-            // deserialize bytes into object
-            ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(serializedObj));
-            return (Serializable) ois.readObject();
-        } catch (final IOException ioe) {
-            throw new RuntimeException(ioe);
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/pom.xml
----------------------------------------------------------------------
diff --git a/commons/pom.xml b/commons/pom.xml
deleted file mode 100644
index 19a7ae0..0000000
--- a/commons/pom.xml
+++ /dev/null
@@ -1,44 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-  
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-
-    <groupId>org.apache.nifi</groupId>
-    <artifactId>nifi-commons-parent</artifactId>
-    <packaging>pom</packaging>
-    <name>NiFi :: Commons Parent</name>
-
-    <modules>
-        <module>data-provenance-utils</module>
-        <module>flowfile-packager</module>
-        <module>nifi-expression-language</module>
-        <module>nifi-logging-utils</module>
-        <module>nifi-properties</module>
-        <module>nifi-security-utils</module>
-        <module>nifi-socket-utils</module>
-        <module>nifi-utils</module>
-        <module>nifi-web-utils</module>
-        <module>processor-utilities</module>
-        <module>wali</module>
-    </modules>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/processor-utilities/pom.xml
----------------------------------------------------------------------
diff --git a/commons/processor-utilities/pom.xml b/commons/processor-utilities/pom.xml
deleted file mode 100644
index 2e952de..0000000
--- a/commons/processor-utilities/pom.xml
+++ /dev/null
@@ -1,45 +0,0 @@
-<?xml version="1.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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-commons-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-    
-    <artifactId>nifi-processor-utils</artifactId>
-    <version>0.0.1-SNAPSHOT</version>
-    <packaging>jar</packaging>
-    <name>NiFi Processor Utils</name>
-    
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-api</artifactId>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-security-utils</artifactId>
-        </dependency>
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/FlowFileFilters.java
----------------------------------------------------------------------
diff --git a/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/FlowFileFilters.java b/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/FlowFileFilters.java
deleted file mode 100644
index 1f77093..0000000
--- a/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/FlowFileFilters.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.processor.util;
-
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.FlowFileFilter;
-
-public class FlowFileFilters {
-
-    /**
-     * Returns a new {@link FlowFileFilter} that will pull FlowFiles until the
-     * maximum file size has been reached, or the maximum FlowFile Count was
-     * been reached (this is important because FlowFiles may be 0 bytes!). If
-     * the first FlowFile exceeds the max size, the FlowFile will be selected
-     * and no other FlowFile will be.
-     *
-     * @param maxSize the maximum size of the group of FlowFiles
-     * @param unit the unit of the <code>maxSize</code> argument
-     * @param maxCount the maximum number of FlowFiles to pull
-     * @return
-     */
-    public static FlowFileFilter newSizeBasedFilter(final double maxSize, final DataUnit unit, final int maxCount) {
-        final double maxBytes = DataUnit.B.convert(maxSize, unit);
-
-        return new FlowFileFilter() {
-            int count = 0;
-            long size = 0L;
-
-            @Override
-            public FlowFileFilterResult filter(final FlowFile flowFile) {
-                if (count == 0) {
-                    count++;
-                    size += flowFile.getSize();
-
-                    return FlowFileFilterResult.ACCEPT_AND_CONTINUE;
-                }
-
-                if ((size + flowFile.getSize() > maxBytes) || (count + 1 > maxCount)) {
-                    return FlowFileFilterResult.REJECT_AND_TERMINATE;
-                }
-
-                count++;
-                size += flowFile.getSize();
-                return FlowFileFilterResult.ACCEPT_AND_CONTINUE;
-            }
-
-        };
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/SSLProperties.java
----------------------------------------------------------------------
diff --git a/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/SSLProperties.java b/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/SSLProperties.java
deleted file mode 100644
index 0d66df5..0000000
--- a/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/SSLProperties.java
+++ /dev/null
@@ -1,226 +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.util;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.MalformedURLException;
-import java.security.KeyManagementException;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import javax.net.ssl.SSLContext;
-
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.security.util.CertificateUtils;
-import org.apache.nifi.security.util.KeystoreType;
-import org.apache.nifi.security.util.SslContextFactory;
-import org.apache.nifi.security.util.SslContextFactory.ClientAuth;
-
-public class SSLProperties {
-
-    public static final PropertyDescriptor TRUSTSTORE = new PropertyDescriptor.Builder()
-            .name("Truststore Filename")
-            .description("The fully-qualified filename of the Truststore")
-            .defaultValue(null)
-            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
-            .sensitive(false)
-            .build();
-
-    public static final PropertyDescriptor TRUSTSTORE_TYPE = new PropertyDescriptor.Builder()
-            .name("Truststore Type")
-            .description("The Type of the Truststore. Either JKS or PKCS12")
-            .allowableValues("JKS", "PKCS12")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .defaultValue(null)
-            .sensitive(false)
-            .build();
-
-    public static final PropertyDescriptor TRUSTSTORE_PASSWORD = new PropertyDescriptor.Builder()
-            .name("Truststore Password")
-            .description("The password for the Truststore")
-            .defaultValue(null)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .sensitive(true)
-            .build();
-
-    public static final PropertyDescriptor KEYSTORE = new PropertyDescriptor.Builder()
-            .name("Keystore Filename")
-            .description("The fully-qualified filename of the Keystore")
-            .defaultValue(null)
-            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
-            .sensitive(false)
-            .build();
-
-    public static final PropertyDescriptor KEYSTORE_TYPE = new PropertyDescriptor.Builder()
-            .name("Keystore Type")
-            .description("The Type of the Keystore")
-            .allowableValues("JKS", "PKCS12")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .sensitive(false)
-            .build();
-
-    public static final PropertyDescriptor KEYSTORE_PASSWORD = new PropertyDescriptor.Builder()
-            .name("Keystore Password")
-            .defaultValue(null)
-            .description("The password for the Keystore")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .sensitive(true)
-            .build();
-
-    public static Collection<ValidationResult> validateStore(final Map<PropertyDescriptor, String> properties) {
-        final Collection<ValidationResult> results = new ArrayList<>();
-        results.addAll(validateStore(properties, KeystoreValidationGroup.KEYSTORE));
-        results.addAll(validateStore(properties, KeystoreValidationGroup.TRUSTSTORE));
-        return results;
-    }
-
-    public static Collection<ValidationResult> validateStore(final Map<PropertyDescriptor, String> properties, final KeystoreValidationGroup keyStoreOrTrustStore) {
-        final Collection<ValidationResult> results = new ArrayList<>();
-
-        final String filename;
-        final String password;
-        final String type;
-
-        if (keyStoreOrTrustStore == KeystoreValidationGroup.KEYSTORE) {
-            filename = properties.get(KEYSTORE);
-            password = properties.get(KEYSTORE_PASSWORD);
-            type = properties.get(KEYSTORE_TYPE);
-        } else {
-            filename = properties.get(TRUSTSTORE);
-            password = properties.get(TRUSTSTORE_PASSWORD);
-            type = properties.get(TRUSTSTORE_TYPE);
-        }
-
-        final String keystoreDesc = (keyStoreOrTrustStore == KeystoreValidationGroup.KEYSTORE) ? "Keystore" : "Truststore";
-
-        final int nulls = countNulls(filename, password, type);
-        if (nulls != 3 && nulls != 0) {
-            results.add(new ValidationResult.Builder().valid(false).explanation("Must set either 0 or 3 properties for " + keystoreDesc).subject(keystoreDesc + " Properties").build());
-        } else if (nulls == 0) {
-            // all properties were filled in.
-            final File file = new File(filename);
-            if (!file.exists() || !file.canRead()) {
-                results.add(new ValidationResult.Builder().valid(false).subject(keystoreDesc + " Properties").explanation("Cannot access file " + file.getAbsolutePath()).build());
-            } else {
-                try {
-                    final boolean storeValid = CertificateUtils.isStoreValid(file.toURI().toURL(), KeystoreType.valueOf(type), password.toCharArray());
-                    if (!storeValid) {
-                        results.add(new ValidationResult.Builder().subject(keystoreDesc + " Properties").valid(false).explanation("Invalid KeyStore Password or Type specified for file " + filename).build());
-                    }
-                } catch (MalformedURLException e) {
-                    results.add(new ValidationResult.Builder().subject(keystoreDesc + " Properties").valid(false).explanation("Malformed URL from file: " + e).build());
-                }
-            }
-        }
-
-        return results;
-    }
-
-    private static int countNulls(Object... objects) {
-        int count = 0;
-        for (final Object x : objects) {
-            if (x == null) {
-                count++;
-            }
-        }
-
-        return count;
-    }
-
-    public static enum KeystoreValidationGroup {
-
-        KEYSTORE, TRUSTSTORE
-    }
-
-    public static List<PropertyDescriptor> getKeystoreDescriptors(final boolean required) {
-        final List<PropertyDescriptor> descriptors = new ArrayList<>();
-        for (final PropertyDescriptor descriptor : KEYSTORE_DESCRIPTORS) {
-            final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder().fromPropertyDescriptor(descriptor).required(required);
-            if (required && descriptor.getName().equals(KEYSTORE_TYPE.getName())) {
-                builder.defaultValue("JKS");
-            }
-            descriptors.add(builder.build());
-        }
-
-        return descriptors;
-    }
-
-    public static List<PropertyDescriptor> getTruststoreDescriptors(final boolean required) {
-        final List<PropertyDescriptor> descriptors = new ArrayList<>();
-        for (final PropertyDescriptor descriptor : TRUSTSTORE_DESCRIPTORS) {
-            final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder().fromPropertyDescriptor(descriptor).required(required);
-            if (required && descriptor.getName().equals(TRUSTSTORE_TYPE.getName())) {
-                builder.defaultValue("JKS");
-            }
-            descriptors.add(builder.build());
-        }
-
-        return descriptors;
-    }
-
-    public static SSLContext createSSLContext(final ProcessContext context, final ClientAuth clientAuth)
-            throws UnrecoverableKeyException, KeyManagementException, KeyStoreException, NoSuchAlgorithmException, CertificateException, IOException {
-        final String keystoreFile = context.getProperty(KEYSTORE).getValue();
-        if (keystoreFile == null) {
-            return SslContextFactory.createTrustSslContext(
-                    context.getProperty(TRUSTSTORE).getValue(),
-                    context.getProperty(TRUSTSTORE_PASSWORD).getValue().toCharArray(),
-                    context.getProperty(TRUSTSTORE_TYPE).getValue());
-        } else {
-            final String truststoreFile = context.getProperty(TRUSTSTORE).getValue();
-            if (truststoreFile == null) {
-                return SslContextFactory.createSslContext(
-                        context.getProperty(KEYSTORE).getValue(),
-                        context.getProperty(KEYSTORE_PASSWORD).getValue().toCharArray(),
-                        context.getProperty(KEYSTORE_TYPE).getValue());
-            } else {
-                return SslContextFactory.createSslContext(
-                        context.getProperty(KEYSTORE).getValue(),
-                        context.getProperty(KEYSTORE_PASSWORD).getValue().toCharArray(),
-                        context.getProperty(KEYSTORE_TYPE).getValue(),
-                        context.getProperty(TRUSTSTORE).getValue(),
-                        context.getProperty(TRUSTSTORE_PASSWORD).getValue().toCharArray(),
-                        context.getProperty(TRUSTSTORE_TYPE).getValue(),
-                        clientAuth);
-            }
-        }
-    }
-
-    private static final Set<PropertyDescriptor> KEYSTORE_DESCRIPTORS = new HashSet<>();
-    private static final Set<PropertyDescriptor> TRUSTSTORE_DESCRIPTORS = new HashSet<>();
-
-    static {
-        KEYSTORE_DESCRIPTORS.add(KEYSTORE);
-        KEYSTORE_DESCRIPTORS.add(KEYSTORE_TYPE);
-        KEYSTORE_DESCRIPTORS.add(KEYSTORE_PASSWORD);
-
-        TRUSTSTORE_DESCRIPTORS.add(TRUSTSTORE);
-        TRUSTSTORE_DESCRIPTORS.add(TRUSTSTORE_TYPE);
-        TRUSTSTORE_DESCRIPTORS.add(TRUSTSTORE_PASSWORD);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java
----------------------------------------------------------------------
diff --git a/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java b/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java
deleted file mode 100644
index 10748fe..0000000
--- a/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java
+++ /dev/null
@@ -1,544 +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.util;
-
-import java.io.File;
-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;
-import org.apache.nifi.util.FormatUtils;
-
-public class StandardValidators {
-
-    //
-    //
-    // STATICALLY DEFINED VALIDATORS
-    //
-    //
-    public static final Validator ATTRIBUTE_KEY_VALIDATOR = new Validator() {
-        @Override
-        public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
-            final ValidationResult.Builder builder = new ValidationResult.Builder();
-            builder.subject(subject).input(input);
-
-            try {
-                FlowFile.KeyValidator.validateKey(input);
-                builder.valid(true);
-            } catch (final IllegalArgumentException e) {
-                builder.valid(false).explanation(e.getMessage());
-            }
-
-            return builder.build();
-        }
-    };
-
-    public static final Validator ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR = new Validator() {
-        @Override
-        public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
-            final ValidationResult.Builder builder = new ValidationResult.Builder();
-            builder.subject("Property Name").input(subject);
-
-            try {
-                FlowFile.KeyValidator.validateKey(subject);
-                builder.valid(true);
-            } catch (final IllegalArgumentException e) {
-                builder.valid(false).explanation(e.getMessage());
-            }
-
-            return builder.build();
-        }
-    };
-
-    public static final Validator POSITIVE_INTEGER_VALIDATOR = new Validator() {
-        @Override
-        public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
-            String reason = null;
-            try {
-                final int intVal = Integer.parseInt(value);
-
-                if (intVal <= 0) {
-                    reason = "not a positive value";
-                }
-            } catch (final NumberFormatException e) {
-                reason = "not a valid integer";
-            }
-
-            return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build();
-        }
-    };
-
-    public static final Validator POSITIVE_LONG_VALIDATOR = new Validator() {
-        @Override
-        public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
-            String reason = null;
-            try {
-                final long longVal = Long.parseLong(value);
-
-                if (longVal <= 0) {
-                    reason = "not a positive value";
-                }
-            } catch (final NumberFormatException e) {
-                reason = "not a valid 64-bit integer";
-            }
-
-            return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build();
-        }
-    };
-
-    public static final Validator PORT_VALIDATOR = createLongValidator(1, 65535, true);
-
-    public static final Validator NON_EMPTY_VALIDATOR = new Validator() {
-        @Override
-        public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
-            return new ValidationResult.Builder().subject(subject).input(value).valid(value != null && !value.isEmpty()).explanation(subject + " cannot be empty").build();
-        }
-    };
-
-    public static final Validator BOOLEAN_VALIDATOR = new Validator() {
-        @Override
-        public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
-            final boolean valid = "true".equalsIgnoreCase(value) || "false".equalsIgnoreCase(value);
-            final String explanation = valid ? null : "Value must be 'true' or 'false'";
-            return new ValidationResult.Builder().subject(subject).input(value).valid(valid).explanation(explanation).build();
-        }
-    };
-
-    public static final Validator INTEGER_VALIDATOR = new Validator() {
-        @Override
-        public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
-            String reason = null;
-            try {
-                Integer.parseInt(value);
-            } catch (final NumberFormatException e) {
-                reason = "not a valid integer";
-            }
-
-            return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build();
-        }
-    };
-
-    public static final Validator LONG_VALIDATOR = new Validator() {
-        @Override
-        public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
-            String reason = null;
-            try {
-                Long.parseLong(value);
-            } catch (final NumberFormatException e) {
-                reason = "not a valid Long";
-            }
-
-            return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build();
-        }
-    };
-
-    public static final Validator NON_NEGATIVE_INTEGER_VALIDATOR = new Validator() {
-        @Override
-        public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
-            String reason = null;
-            try {
-                final int intVal = Integer.parseInt(value);
-
-                if (intVal < 0) {
-                    reason = "value is negative";
-                }
-            } catch (final NumberFormatException e) {
-                reason = "value is not a valid integer";
-            }
-
-            return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build();
-        }
-    };
-
-    public static final Validator CHARACTER_SET_VALIDATOR = new Validator() {
-        @Override
-        public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
-            String reason = null;
-            try {
-                if (!Charset.isSupported(value)) {
-                    reason = "Character Set is not supported by this JVM.";
-                }
-            } catch (final UnsupportedCharsetException uce) {
-                reason = "Character Set is not supported by this JVM.";
-            } catch (final IllegalArgumentException iae) {
-                reason = "Character Set value cannot be null.";
-            }
-
-            return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build();
-        }
-    };
-
-    /**
-     * URL Validator that does not allow the Expression Language to be used
-     */
-    public static final Validator URL_VALIDATOR = createURLValidator();
-
-    public static final Validator URI_VALIDATOR = new Validator() {
-        @Override
-        public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
-            try {
-                new URI(input);
-                return new ValidationResult.Builder().subject(subject).input(input).explanation("Valid URI").valid(true).build();
-            } catch (final Exception e) {
-                return new ValidationResult.Builder().subject(subject).input(input).explanation("Not a valid URI").valid(false).build();
-            }
-        }
-    };
-
-    public static final Validator REGULAR_EXPRESSION_VALIDATOR = createRegexValidator(0, Integer.MAX_VALUE, false);
-
-    public static final Validator ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR = new Validator() {
-        @Override
-        public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
-            try {
-                context.newExpressionLanguageCompiler().compile(input);
-                return new ValidationResult.Builder().subject(subject).input(input).valid(true).build();
-            } catch (final Exception e) {
-                return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation(e.getMessage()).build();
-            }
-        }
-
-    };
-
-    public static final Validator TIME_PERIOD_VALIDATOR = new Validator() {
-        @Override
-        public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
-            if (input == null) {
-                return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Time Period cannot be null").build();
-            }
-            if (Pattern.compile(FormatUtils.TIME_DURATION_REGEX).matcher(input.toLowerCase()).matches()) {
-                return new ValidationResult.Builder().subject(subject).input(input).valid(true).build();
-            } else {
-                return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Must be of format <duration> <TimeUnit> where <duration> is a non-negative integer and TimeUnit is a supported Time Unit, such as: nanos, millis, secs, mins, hrs, days").build();
-            }
-        }
-    };
-
-    public static final Validator DATA_SIZE_VALIDATOR = new Validator() {
-        @Override
-        public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
-            if (input == null) {
-                return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Data Size cannot be null").build();
-            }
-            if (Pattern.compile(DataUnit.DATA_SIZE_REGEX).matcher(input.toUpperCase()).matches()) {
-                return new ValidationResult.Builder().subject(subject).input(input).valid(true).build();
-            } else {
-                return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Must be of format <Data Size> <Data Unit> where <Data Size> is a non-negative integer and <Data Unit> is a supported Data Unit, such as: B, KB, MB, GB, TB").build();
-            }
-        }
-    };
-
-    public static final Validator FILE_EXISTS_VALIDATOR = new FileExistsValidator(true);
-
-    //
-    //
-    // FACTORY METHODS FOR VALIDATORS
-    //
-    //
-    public static Validator createDirectoryExistsValidator(final boolean allowExpressionLanguage, final boolean createDirectoryIfMissing) {
-        return new DirectoryExistsValidator(allowExpressionLanguage, createDirectoryIfMissing);
-    }
-
-    private static Validator createURLValidator() {
-        return new Validator() {
-            @Override
-            public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
-                try {
-                    final String evaluatedInput = context.newPropertyValue(input).evaluateAttributeExpressions().getValue();
-                    new URL(evaluatedInput);
-                    return new ValidationResult.Builder().subject(subject).input(input).explanation("Valid URL").valid(true).build();
-                } catch (final Exception e) {
-                    return new ValidationResult.Builder().subject(subject).input(input).explanation("Not a valid URL").valid(false).build();
-                }
-            }
-        };
-    }
-
-    public static Validator createTimePeriodValidator(final long minTime, final TimeUnit minTimeUnit, final long maxTime, final TimeUnit maxTimeUnit) {
-        return new TimePeriodValidator(minTime, minTimeUnit, maxTime, maxTimeUnit);
-    }
-
-    public static Validator createAttributeExpressionLanguageValidator(final ResultType expectedResultType) {
-        return createAttributeExpressionLanguageValidator(expectedResultType, true);
-    }
-
-    public static Validator createRegexMatchingValidator(final Pattern pattern) {
-        return new Validator() {
-            @Override
-            public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
-                final boolean matches = pattern.matcher(input).matches();
-                return new ValidationResult.Builder()
-                        .input(input)
-                        .subject(subject)
-                        .valid(matches)
-                        .explanation(matches ? null : "Value does not match regular expression: " + pattern.pattern())
-                        .build();
-            }
-        };
-    }
-
-    /**
-     * Creates a @{link Validator} that ensure that a value is a valid Java
-     * Regular Expression with at least <code>minCapturingGroups</code>
-     * capturing groups and at most <code>maxCapturingGroups</code> capturing
-     * groups. If <code>supportAttributeExpressionLanguage</code> is set to
-     * <code>true</code>, the value may also include the Expression Language,
-     * but the result of evaluating the Expression Language will be applied
-     * before the Regular Expression is performed. In this case, the Expression
-     * Language will not support FlowFile Attributes but only System/JVM
-     * Properties
-     *
-     * @param minCapturingGroups
-     * @param maxCapturingGroups
-     * @param supportAttributeExpressionLanguage
-     * @return
-     */
-    public static Validator createRegexValidator(final int minCapturingGroups, final int maxCapturingGroups, final boolean supportAttributeExpressionLanguage) {
-        return new Validator() {
-            @Override
-            public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
-                try {
-                    final String substituted;
-                    if (supportAttributeExpressionLanguage) {
-                        try {
-                            substituted = context.newPropertyValue(value).evaluateAttributeExpressions().getValue();
-                        } catch (final Exception e) {
-                            return new ValidationResult.Builder().subject(subject).input(value).valid(false).explanation("Failed to evaluate the Attribute Expression Language due to " + e.toString()).build();
-                        }
-                    } else {
-                        substituted = value;
-                    }
-
-                    final Pattern pattern = Pattern.compile(substituted);
-                    final int numGroups = pattern.matcher("").groupCount();
-                    if (numGroups < minCapturingGroups || numGroups > maxCapturingGroups) {
-                        return new ValidationResult.Builder().subject(subject).input(value).valid(false).explanation("RegEx is required to have between " + minCapturingGroups + " and " + maxCapturingGroups + " Capturing Groups but has " + numGroups).build();
-                    }
-
-                    return new ValidationResult.Builder().subject(subject).input(value).valid(true).build();
-                } catch (final Exception e) {
-                    return new ValidationResult.Builder().subject(subject).input(value).valid(false).explanation("Not a valid Java Regular Expression").build();
-                }
-
-            }
-        };
-    }
-
-    public static Validator createAttributeExpressionLanguageValidator(final ResultType expectedResultType, final boolean allowExtraCharacters) {
-        return new Validator() {
-            @Override
-            public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
-                final String syntaxError = context.newExpressionLanguageCompiler().validateExpression(input, allowExtraCharacters);
-                if (syntaxError != null) {
-                    return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation(syntaxError).build();
-                }
-
-                final ResultType resultType = allowExtraCharacters ? ResultType.STRING : context.newExpressionLanguageCompiler().getResultType(input);
-                if (!resultType.equals(expectedResultType)) {
-                    return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Expected Attribute Query to return type " + expectedResultType + " but query returns type " + resultType).build();
-                }
-
-                return new ValidationResult.Builder().subject(subject).input(input).valid(true).build();
-            }
-        };
-    }
-
-    public static Validator createLongValidator(final long minimum, final long maximum, final boolean inclusive) {
-        return new Validator() {
-            @Override
-            public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
-                String reason = null;
-                try {
-                    final long longVal = Long.parseLong(input);
-                    if (longVal < minimum || (!inclusive && longVal == minimum) | longVal > maximum || (!inclusive && longVal == maximum)) {
-                        reason = "Value must be between " + minimum + " and " + maximum + " (" + (inclusive ? "inclusive" : "exclusive") + ")";
-                    }
-                } catch (final NumberFormatException e) {
-                    reason = "not a valid integer";
-                }
-
-                return new ValidationResult.Builder().subject(subject).input(input).explanation(reason).valid(reason == null).build();
-            }
-
-        };
-    }
-
-    //
-    //
-    // SPECIFIC VALIDATOR IMPLEMENTATIONS THAT CANNOT BE ANONYMOUS CLASSES
-    //
-    //
-    static class TimePeriodValidator implements Validator {
-
-        private final Pattern pattern;
-
-        private final long minNanos;
-        private final long maxNanos;
-
-        private final String minValueEnglish;
-        private final String maxValueEnglish;
-
-        public TimePeriodValidator(final long minValue, final TimeUnit minTimeUnit, final long maxValue, final TimeUnit maxTimeUnit) {
-            pattern = Pattern.compile(FormatUtils.TIME_DURATION_REGEX);
-
-            this.minNanos = TimeUnit.NANOSECONDS.convert(minValue, minTimeUnit);
-            this.maxNanos = TimeUnit.NANOSECONDS.convert(maxValue, maxTimeUnit);
-            this.minValueEnglish = minValue + " " + minTimeUnit.toString();
-            this.maxValueEnglish = maxValue + " " + maxTimeUnit.toString();
-        }
-
-        @Override
-        public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
-            if (input == null) {
-                return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Time Period cannot be null").build();
-            }
-            final String lowerCase = input.toLowerCase();
-            final boolean validSyntax = pattern.matcher(lowerCase).matches();
-            final ValidationResult.Builder builder = new ValidationResult.Builder();
-            if (validSyntax) {
-                final long nanos = FormatUtils.getTimeDuration(lowerCase, TimeUnit.NANOSECONDS);
-
-                if (nanos < minNanos || nanos > maxNanos) {
-                    builder.subject(subject).input(input).valid(false)
-                            .explanation("Must be in the range of " + minValueEnglish + " to " + maxValueEnglish);
-                } else {
-                    builder.subject(subject).input(input).valid(true);
-                }
-            } else {
-                builder.subject(subject).input(input).valid(false)
-                        .explanation("Must be of format <duration> <TimeUnit> where <duration> is a non-negative integer and TimeUnit is a supported Time Unit, such as: nanos, millis, secs, mins, hrs, days");
-            }
-            return builder.build();
-        }
-    }
-
-    public static class FileExistsValidator implements Validator {
-
-        private final boolean allowEL;
-
-        public FileExistsValidator(final boolean allowExpressionLanguage) {
-            this.allowEL = allowExpressionLanguage;
-        }
-
-        @Override
-        public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
-            final String substituted;
-            if (allowEL) {
-                try {
-                    substituted = context.newPropertyValue(value).evaluateAttributeExpressions().getValue();
-                } catch (final Exception e) {
-                    return new ValidationResult.Builder().subject(subject).input(value).valid(false)
-                            .explanation("Not a valid Expression Language value: " + e.getMessage()).build();
-                }
-            } else {
-                substituted = value;
-            }
-
-            final File file = new File(substituted);
-            final boolean valid = file.exists();
-            final String explanation = valid ? null : "File " + file + " does not exist";
-            return new ValidationResult.Builder().subject(subject).input(value).valid(valid).explanation(explanation).build();
-        }
-    }
-
-    public static class DirectoryExistsValidator implements Validator {
-
-        private final boolean allowEL;
-        private final boolean create;
-
-        public DirectoryExistsValidator(final boolean allowExpressionLanguage, final boolean create) {
-            this.allowEL = allowExpressionLanguage;
-            this.create = create;
-        }
-
-        @Override
-        public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
-            final String substituted;
-            if (allowEL) {
-                try {
-                    substituted = context.newPropertyValue(value).evaluateAttributeExpressions().getValue();
-                } catch (final Exception e) {
-                    return new ValidationResult.Builder().subject(subject).input(value).valid(false)
-                            .explanation("Not a valid Expression Language value: " + e.getMessage()).build();
-                }
-
-                if (substituted.trim().isEmpty() && !value.trim().isEmpty()) {
-                    // User specified an Expression and nothing more... assume valid.
-                    return new ValidationResult.Builder().subject(subject).input(value).valid(true).build();
-                }
-            } else {
-                substituted = value;
-            }
-
-            String reason = null;
-            try {
-                final File file = new File(substituted);
-                if (!file.exists()) {
-                    if (!create) {
-                        reason = "Directory does not exist";
-                    } else if (!file.mkdirs()) {
-                        reason = "Directory does not exist and could not be created";
-                    }
-                } else if (!file.isDirectory()) {
-                    reason = "Path does not point to a directory";
-                }
-            } catch (final Exception e) {
-                reason = "Value is not a valid directory name";
-            }
-
-            return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build();
-        }
-    }
-
-    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) {
-                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/incubator-nifi/blob/300952a9/commons/processor-utilities/src/test/java/org/apache/nifi/processor/TestFormatUtils.java
----------------------------------------------------------------------
diff --git a/commons/processor-utilities/src/test/java/org/apache/nifi/processor/TestFormatUtils.java b/commons/processor-utilities/src/test/java/org/apache/nifi/processor/TestFormatUtils.java
deleted file mode 100644
index 359def2..0000000
--- a/commons/processor-utilities/src/test/java/org/apache/nifi/processor/TestFormatUtils.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;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.util.FormatUtils;
-
-import org.junit.Test;
-
-public class TestFormatUtils {
-
-    @Test
-    public void testParse() {
-        assertEquals(3, FormatUtils.getTimeDuration("3000 ms", TimeUnit.SECONDS));
-        assertEquals(3000, FormatUtils.getTimeDuration("3000 s", TimeUnit.SECONDS));
-        assertEquals(0, FormatUtils.getTimeDuration("999 millis", TimeUnit.SECONDS));
-        assertEquals(4L * 24L * 60L * 60L * 1000000000L, FormatUtils.getTimeDuration("4 days", TimeUnit.NANOSECONDS));
-        assertEquals(24, FormatUtils.getTimeDuration("1 DAY", TimeUnit.HOURS));
-        assertEquals(60, FormatUtils.getTimeDuration("1 hr", TimeUnit.MINUTES));
-        assertEquals(60, FormatUtils.getTimeDuration("1 Hrs", TimeUnit.MINUTES));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/processor-utilities/src/test/java/org/apache/nifi/processor/util/TestStandardValidators.java
----------------------------------------------------------------------
diff --git a/commons/processor-utilities/src/test/java/org/apache/nifi/processor/util/TestStandardValidators.java b/commons/processor-utilities/src/test/java/org/apache/nifi/processor/util/TestStandardValidators.java
deleted file mode 100644
index 2ae50c9..0000000
--- a/commons/processor-utilities/src/test/java/org/apache/nifi/processor/util/TestStandardValidators.java
+++ /dev/null
@@ -1,54 +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.util;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.components.Validator;
-
-import org.junit.Test;
-
-public class TestStandardValidators {
-
-    @Test
-    public void testTimePeriodValidator() {
-        Validator val = StandardValidators.createTimePeriodValidator(1L, TimeUnit.SECONDS, Long.MAX_VALUE, TimeUnit.NANOSECONDS);
-        ValidationResult vr;
-
-        vr = val.validate("TimePeriodTest", "0 sense made", null);
-        assertFalse(vr.isValid());
-
-        vr = val.validate("TimePeriodTest", null, null);
-        assertFalse(vr.isValid());
-
-        vr = val.validate("TimePeriodTest", "0 secs", null);
-        assertFalse(vr.isValid());
-
-        vr = val.validate("TimePeriodTest", "999 millis", null);
-        assertFalse(vr.isValid());
-
-        vr = val.validate("TimePeriodTest", "999999999 nanos", null);
-        assertFalse(vr.isValid());
-
-        vr = val.validate("TimePeriodTest", "1 sec", null);
-        assertTrue(vr.isValid());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/commons/wali/.gitignore
----------------------------------------------------------------------
diff --git a/commons/wali/.gitignore b/commons/wali/.gitignore
deleted file mode 100755
index 19f2e00..0000000
--- a/commons/wali/.gitignore
+++ /dev/null
@@ -1,2 +0,0 @@
-/target
-/target


[03/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowService.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
deleted file mode 100644
index d459b00..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
+++ /dev/null
@@ -1,875 +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;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.InetSocketAddress;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.nio.file.StandardOpenOption;
-import java.util.Calendar;
-import java.util.Date;
-import java.util.UUID;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.zip.GZIPInputStream;
-import java.util.zip.GZIPOutputStream;
-
-import org.apache.nifi.cluster.ConnectionException;
-import org.apache.nifi.cluster.protocol.ConnectionRequest;
-import org.apache.nifi.cluster.protocol.ConnectionResponse;
-import org.apache.nifi.cluster.protocol.DataFlow;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.StandardDataFlow;
-import org.apache.nifi.cluster.protocol.UnknownServiceAddressException;
-import org.apache.nifi.cluster.protocol.impl.NodeProtocolSenderListener;
-import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage;
-import org.apache.nifi.cluster.protocol.message.ControllerStartupFailureMessage;
-import org.apache.nifi.cluster.protocol.message.DisconnectMessage;
-import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
-import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
-import org.apache.nifi.cluster.protocol.message.PrimaryRoleAssignmentMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.ReconnectionFailureMessage;
-import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
-import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage;
-import org.apache.nifi.engine.FlowEngine;
-import org.apache.nifi.events.BulletinFactory;
-import org.apache.nifi.util.file.FileUtils;
-import org.apache.nifi.groups.ProcessGroup;
-import org.apache.nifi.lifecycle.LifeCycleStartException;
-import org.apache.nifi.logging.LogLevel;
-import org.apache.nifi.persistence.FlowConfigurationDAO;
-import org.apache.nifi.persistence.StandardXMLFlowConfigurationDAO;
-import org.apache.nifi.reporting.Bulletin;
-import org.apache.nifi.services.FlowService;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.NiFiProperties;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.encrypt.StringEncryptor;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class StandardFlowService implements FlowService, ProtocolHandler {
-
-    private static final String EVENT_CATEGORY = "Controller";
-
-    private final FlowController controller;
-    private final Path flowXml;
-    private final Path taskConfigXml;
-    private final Path serviceConfigXml;
-    private final FlowConfigurationDAO dao;
-    private final int gracefulShutdownSeconds;
-    private final boolean autoResumeState;
-    private final int connectionRetryMillis;
-    private final StringEncryptor encryptor;
-
-    // Lock is used to protect the flow.xml file.
-    private final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
-    private final Lock readLock = rwLock.readLock();
-    private final Lock writeLock = rwLock.writeLock();
-
-    private final AtomicBoolean running = new AtomicBoolean(false);
-    private final AtomicReference<ScheduledExecutorService> executor = new AtomicReference<>(null);
-    private final AtomicReference<SaveHolder> saveHolder = new AtomicReference<>(null);
-
-    /**
-     * listener/sender for internal cluster communication
-     */
-    private final NodeProtocolSenderListener senderListener;
-
-    /**
-     * flag indicating whether we are operating in a clustered environment
-     */
-    private final boolean configuredForClustering;
-
-    /**
-     * the node identifier
-     */
-    private NodeIdentifier nodeId;
-
-    // guardedBy rwLock
-    private boolean firstControllerInitialization = true;
-
-    private static final String CONNECTION_EXCEPTION_MSG_PREFIX = "Failed to connect node to cluster because ";
-    private static final Logger logger = LoggerFactory.getLogger(StandardFlowService.class);
-
-    public static StandardFlowService createStandaloneInstance(
-            final FlowController controller,
-            final NiFiProperties properties,
-            final StringEncryptor encryptor) throws IOException {
-        return new StandardFlowService(
-                controller,
-                properties,
-                /* nodeProtocolSenderListener */ null,
-                encryptor,
-                /* configuredForClustering */ false);
-    }
-
-    public static StandardFlowService createClusteredInstance(
-            final FlowController controller,
-            final NiFiProperties properties,
-            final NodeProtocolSenderListener senderListener,
-            final StringEncryptor encryptor) throws IOException {
-        return new StandardFlowService(
-                controller,
-                properties,
-                senderListener,
-                encryptor,
-                /* configuredForClustering */ true);
-    }
-
-    private StandardFlowService(
-            final FlowController controller,
-            final NiFiProperties properties,
-            final NodeProtocolSenderListener senderListener,
-            final StringEncryptor encryptor,
-            final boolean configuredForClustering) throws IOException {
-
-        this.controller = controller;
-        this.encryptor = encryptor;
-        flowXml = Paths.get(properties.getProperty(NiFiProperties.FLOW_CONFIGURATION_FILE));
-        taskConfigXml = Paths.get(properties.getProperty(NiFiProperties.TASK_CONFIGURATION_FILE));
-        serviceConfigXml = Paths.get(properties.getProperty(NiFiProperties.SERVICE_CONFIGURATION_FILE));
-
-        gracefulShutdownSeconds = (int) FormatUtils.getTimeDuration(properties.getProperty(NiFiProperties.FLOW_CONTROLLER_GRACEFUL_SHUTDOWN_PERIOD), TimeUnit.SECONDS);
-        autoResumeState = properties.getAutoResumeState();
-        connectionRetryMillis = (int) FormatUtils.getTimeDuration(properties.getClusterManagerFlowRetrievalDelay(), TimeUnit.MILLISECONDS);
-
-        dao = new StandardXMLFlowConfigurationDAO(flowXml, taskConfigXml, serviceConfigXml, encryptor);
-
-        if (configuredForClustering) {
-
-            this.configuredForClustering = configuredForClustering;
-
-            this.senderListener = senderListener;
-            senderListener.addHandler(this);
-
-            final InetSocketAddress nodeApiAddress = properties.getNodeApiAddress();
-            final InetSocketAddress nodeSocketAddress = properties.getClusterNodeProtocolAddress();
-
-            // use a random UUID as the proposed node identifier
-            this.nodeId = new NodeIdentifier(UUID.randomUUID().toString(), nodeApiAddress.getHostName(), nodeApiAddress.getPort(), nodeSocketAddress.getHostName(), nodeSocketAddress.getPort());
-
-        } else {
-            this.configuredForClustering = false;
-            this.senderListener = null;
-        }
-
-    }
-
-    @Override
-    public void saveFlowChanges() throws IOException {
-        writeLock.lock();
-        try {
-            dao.save(controller);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public void saveFlowChanges(final OutputStream outStream) throws IOException {
-        writeLock.lock();
-        try {
-            dao.save(controller, outStream);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public void overwriteFlow(final InputStream is) throws IOException {
-        writeLock.lock();
-        try (final OutputStream output = Files.newOutputStream(flowXml, StandardOpenOption.WRITE, StandardOpenOption.CREATE);
-                final OutputStream gzipOut = new GZIPOutputStream(output);) {
-            FileUtils.copy(is, gzipOut);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public void saveFlowChanges(final TimeUnit delayUnit, final long delay) {
-        saveFlowChanges(delayUnit, delay, false);
-    }
-
-    @Override
-    public void saveFlowChanges(final TimeUnit delayUnit, final long delay, final boolean archive) {
-        final Calendar saveTime = Calendar.getInstance();
-        final long delayInMs = TimeUnit.MILLISECONDS.convert(delay, delayUnit);
-        int finalDelayMs = 500; //default to 500 ms.
-        if (delayInMs <= Integer.MAX_VALUE) {
-            finalDelayMs = (int) delayInMs;
-        }
-        saveTime.add(Calendar.MILLISECOND, finalDelayMs);
-
-        if (logger.isTraceEnabled()) {
-            logger.trace(" A request to save the flow has been made with delay {} for time {}", finalDelayMs, saveTime.getTime());
-        }
-
-        saveHolder.set(new SaveHolder(saveTime, archive));
-    }
-
-    @Override
-    public boolean isRunning() {
-        return running.get();
-    }
-
-    @Override
-    public void start() throws LifeCycleStartException {
-        writeLock.lock();
-        try {
-
-            if (isRunning()) {
-                return;
-            }
-
-            running.set(true);
-
-            final ScheduledExecutorService newExecutor = new FlowEngine(2, "Flow Service Tasks");
-            newExecutor.scheduleWithFixedDelay(new SaveReportingTask(), 0L, 500L, TimeUnit.MILLISECONDS);
-            this.executor.set(newExecutor);
-
-            if (configuredForClustering) {
-                senderListener.start();
-            }
-
-        } catch (final IOException ioe) {
-            try {
-                stop(/* force */true);
-            } catch (final Exception e) {
-            }
-
-            throw new LifeCycleStartException("Failed to start Flow Service due to: " + ioe, ioe);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public void stop(final boolean force) {
-        writeLock.lock();
-        try {
-
-            if (!isRunning()) {
-                return;
-            }
-
-            running.set(false);
-
-            if (!controller.isTerminated()) {
-                controller.shutdown(force);
-            }
-
-            if (configuredForClustering && senderListener != null) {
-                try {
-                    senderListener.stop();
-                } catch (final IOException ioe) {
-                    logger.warn("Protocol sender/listener did not stop gracefully due to: " + ioe);
-                }
-            }
-
-            final ScheduledExecutorService executorService = executor.get();
-            if (executorService != null) {
-                if (force) {
-                    executorService.shutdownNow();
-                } else {
-                    executorService.shutdown();
-                }
-
-                boolean graceful;
-                try {
-                    graceful = executorService.awaitTermination(gracefulShutdownSeconds, TimeUnit.SECONDS);
-                } catch (final InterruptedException e) {
-                    graceful = false;
-                }
-
-                if (!graceful) {
-                    logger.warn("Scheduling service did not gracefully shutdown within configured " + gracefulShutdownSeconds + " second window");
-                }
-            }
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public boolean canHandle(final ProtocolMessage msg) {
-        switch (msg.getType()) {
-            case RECONNECTION_REQUEST:
-            case DISCONNECTION_REQUEST:
-            case FLOW_REQUEST:
-            case PRIMARY_ROLE:
-                return true;
-            default:
-                return false;
-        }
-    }
-
-    @Override
-    public ProtocolMessage handle(final ProtocolMessage request) throws ProtocolException {
-        final long startNanos = System.nanoTime();
-        try {
-            switch (request.getType()) {
-                case FLOW_REQUEST:
-                    return handleFlowRequest((FlowRequestMessage) request);
-                case RECONNECTION_REQUEST:
-                    // Suspend heartbeats until we've reconnected. Otherwise,
-                    // we may send a heartbeat while we are still in the process of
-                    // connecting, which will cause the Cluster Manager to mark us 
-                    // as "Connected," which becomes problematic as the FlowController's lock
-                    // may still be held, causing this node to take a long time to respond to requests.
-                    controller.suspendHeartbeats();
-
-                    new Thread(new Runnable() {
-                        @Override
-                        public void run() {
-                            handleReconnectionRequest((ReconnectionRequestMessage) request);
-                        }
-                    }, "Reconnect to Cluster").start();
-
-                    return new ReconnectionResponseMessage();
-                case DISCONNECTION_REQUEST:
-                    new Thread(new Runnable() {
-                        @Override
-                        public void run() {
-                            handleDisconnectionRequest((DisconnectMessage) request);
-                        }
-                    }, "Disconnect from Cluster").start();
-
-                    return null;
-                case PRIMARY_ROLE:
-                    new Thread(new Runnable() {
-                        @Override
-                        public void run() {
-                            handlePrimaryRoleAssignment((PrimaryRoleAssignmentMessage) request);
-                        }
-                    }, "Set Primary Role Status").start();
-                    return null;
-                default:
-                    throw new ProtocolException("Handler cannot handle message type: " + request.getType());
-            }
-        } finally {
-            if (logger.isDebugEnabled()) {
-                final long procNanos = System.nanoTime() - startNanos;
-                final long procMillis = TimeUnit.MILLISECONDS.convert(procNanos, TimeUnit.NANOSECONDS);
-                logger.debug("Finished Processing Protocol Message of type {} in {} millis", request.getType(), procMillis);
-            }
-        }
-    }
-
-    @Override
-    public void load(final DataFlow proposedFlow) throws IOException, FlowSerializationException,
-            FlowSynchronizationException, UninheritableFlowException {
-        writeLock.lock();
-        try {
-            if (configuredForClustering) {
-                /*
-                 * Attempt to connect to the cluster.  If the manager is able to
-                 * provide a data flow, then the manager will send a connection
-                 * response.  If the manager was unable to be located, then 
-                 * the response will be null and we should load the local dataflow
-                 * and heartbeat until a manager is located.
-                 */
-                final boolean localFlowEmpty = StandardFlowSynchronizer.isEmpty(proposedFlow, encryptor);
-                final ConnectionResponse response = connect(localFlowEmpty, localFlowEmpty);
-
-                if (response == null) {
-                    logger.info("Flow controller will load local dataflow and suspend connection handshake until a cluster connection response is received.");
-
-                    // load local proposed flow
-                    loadFromBytes(proposedFlow, false);
-
-                    // set node ID on controller before we start heartbeating because heartbeat needs node ID
-                    controller.setNodeId(nodeId);
-
-                    // set node as clustered, since it is trying to connect to a cluster
-                    controller.setClustered(true, null);
-                    controller.setClusterManagerRemoteSiteInfo(null, null);
-                    controller.setConnected(false);
-
-                    /*
-                     * Start heartbeating.  Heartbeats will fail because we can't reach 
-                     * the manager, but when we locate the manager, the node will 
-                     * reconnect and establish a connection to the cluster.  The 
-                     * heartbeat is the trigger that will cause the manager to 
-                     * issue a reconnect request.
-                     */
-                    controller.startHeartbeating();
-
-                    // if configured, start all components
-                    if (autoResumeState) {
-                        try {
-                            controller.startDelayed();
-                        } catch (final Exception ex) {
-                            logger.warn("Unable to start all processors due to invalid flow configuration.");
-                            if (logger.isDebugEnabled()) {
-                                logger.warn(StringUtils.EMPTY, ex);
-                            }
-                        }
-                    }
-
-                } else {
-                    try {
-                        loadFromConnectionResponse(response);
-                    } catch (final ConnectionException ce) {
-                        logger.error("Failed to load flow from cluster due to: " + ce, ce);
-
-                        /*
-                         * If we failed processing the response, then we want to notify
-                         * the manager so that it can mark the node as disconnected.
-                         */
-                        // create error message
-                        final ControllerStartupFailureMessage msg = new ControllerStartupFailureMessage();
-                        msg.setExceptionMessage(ce.getMessage());
-                        msg.setNodeId(response.getNodeIdentifier());
-
-                        // send error message to manager
-                        try {
-                            senderListener.notifyControllerStartupFailure(msg);
-                        } catch (final ProtocolException | UnknownServiceAddressException e) {
-                            logger.warn("Failed to notify cluster manager of controller startup failure due to: " + e, e);
-                        }
-
-                        throw new IOException(ce);
-                    }
-                }
-            } else {
-                // operating in standalone mode, so load proposed flow
-                loadFromBytes(proposedFlow, true);
-            }
-
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    private FlowResponseMessage handleFlowRequest(final FlowRequestMessage request) throws ProtocolException {
-        readLock.lock();
-        try {
-            logger.info("Received flow request message from manager.");
-
-            // serialize the flow to the output stream
-            final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-            copyCurrentFlow(baos);
-            final byte[] flowBytes = baos.toByteArray();
-            baos.reset();
-
-            final byte[] templateBytes = controller.getTemplateManager().export();
-            final byte[] snippetBytes = controller.getSnippetManager().export();
-
-            // create the response
-            final FlowResponseMessage response = new FlowResponseMessage();
-
-            response.setDataFlow(new StandardDataFlow(flowBytes, templateBytes, snippetBytes));
-
-            return response;
-
-        } catch (final Exception ex) {
-            throw new ProtocolException("Failed serializing flow controller state for flow request due to: " + ex, ex);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    private void handlePrimaryRoleAssignment(final PrimaryRoleAssignmentMessage msg) {
-        writeLock.lock();
-        try {
-            controller.setPrimary(msg.isPrimary());
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    private void handleReconnectionRequest(final ReconnectionRequestMessage request) {
-        writeLock.lock();
-        try {
-            logger.info("Processing reconnection request from manager.");
-
-            // reconnect
-            final ConnectionResponse connectionResponse = new ConnectionResponse(nodeId, request.getDataFlow(), request.isPrimary(),
-                    request.getManagerRemoteSiteListeningPort(), request.isManagerRemoteSiteCommsSecure(), request.getInstanceId());
-            connectionResponse.setClusterManagerDN(request.getRequestorDN());
-            loadFromConnectionResponse(connectionResponse);
-
-            controller.resumeHeartbeats();  // we are now connected, so resume sending heartbeats.
-
-            logger.info("Node reconnected.");
-        } catch (final Exception ex) {
-            // disconnect controller 
-            if (controller.isClustered()) {
-                disconnect();
-            }
-
-            logger.error("Handling reconnection request failed due to: " + ex, ex);
-
-            final ReconnectionFailureMessage failureMessage = new ReconnectionFailureMessage();
-            failureMessage.setNodeId(request.getNodeId());
-            failureMessage.setExceptionMessage(ex.toString());
-
-            // send error message to manager
-            try {
-                senderListener.notifyReconnectionFailure(failureMessage);
-            } catch (final ProtocolException | UnknownServiceAddressException e) {
-                logger.warn("Failed to notify cluster manager of controller reconnection failure due to: " + e, e);
-            }
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    private void handleDisconnectionRequest(final DisconnectMessage request) {
-        writeLock.lock();
-        try {
-            logger.info("Received disconnection request message from manager with explanation: " + request.getExplanation());
-            disconnect();
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    private void disconnect() {
-        writeLock.lock();
-        try {
-
-            logger.info("Disconnecting node.");
-
-            // mark node as not connected
-            controller.setConnected(false);
-
-            // turn off primary flag
-            controller.setPrimary(false);
-
-            // stop heartbeating
-            controller.stopHeartbeating();
-
-            // set node to not clustered
-            controller.setClustered(false, null);
-
-            logger.info("Node disconnected.");
-
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    // write lock must already be acquired
-    private void loadFromBytes(final DataFlow proposedFlow, final boolean allowEmptyFlow)
-            throws IOException, FlowSerializationException, FlowSynchronizationException, UninheritableFlowException {
-        logger.trace("Loading flow from bytes");
-        final TemplateManager templateManager = controller.getTemplateManager();
-        templateManager.loadTemplates();
-        logger.trace("Finished loading templates");
-
-        // resolve the given flow (null means load flow from disk)
-        final DataFlow actualProposedFlow;
-        final byte[] flowBytes;
-        final byte[] templateBytes;
-        if (proposedFlow == null) {
-            final ByteArrayOutputStream flowOnDisk = new ByteArrayOutputStream();
-            copyCurrentFlow(flowOnDisk);
-            flowBytes = flowOnDisk.toByteArray();
-            templateBytes = templateManager.export();
-            logger.debug("Loaded Flow from bytes");
-        } else {
-            flowBytes = proposedFlow.getFlow();
-            templateBytes = proposedFlow.getTemplates();
-            logger.debug("Loaded flow from proposed flow");
-        }
-
-        actualProposedFlow = new StandardDataFlow(flowBytes, templateBytes, null);
-
-        if (firstControllerInitialization) {
-            // load the controller services
-            logger.debug("Loading controller services");
-            dao.loadControllerServices(controller);
-        }
-
-        // load the flow
-        logger.debug("Loading proposed flow into FlowController");
-        dao.load(controller, actualProposedFlow);
-
-        final ProcessGroup rootGroup = controller.getGroup(controller.getRootGroupId());
-        if (rootGroup.isEmpty() && !allowEmptyFlow) {
-            throw new FlowSynchronizationException("Failed to load flow because unable to connect to cluster and local flow is empty");
-        }
-
-        // lazy initialization of controller tasks and flow
-        if (firstControllerInitialization) {
-            logger.debug("First controller initialization. Loading reporting tasks and initializing controller.");
-
-            // load the controller tasks
-            dao.loadReportingTasks(controller);
-
-            // initialize the flow
-            controller.initializeFlow();
-
-            firstControllerInitialization = false;
-        }
-    }
-
-    private ConnectionResponse connect(final boolean retryOnCommsFailure, final boolean retryIndefinitely) throws ConnectionException {
-        writeLock.lock();
-        try {
-            logger.info("Connecting Node: " + nodeId);
-
-            // create connection request message
-            final ConnectionRequest request = new ConnectionRequest(nodeId);
-            final ConnectionRequestMessage requestMsg = new ConnectionRequestMessage();
-            requestMsg.setConnectionRequest(request);
-
-            // send connection request to cluster manager
-            /*
-             * Try to get a current copy of the cluster's dataflow from the manager 
-             * for ten times, sleeping between attempts.  Ten times should be 
-             * enough because the manager will register the node as connecting
-             * and therefore, no other changes to the cluster flow can occur.
-             * 
-             * However, the manager needs to obtain a current data flow within
-             * maxAttempts * tryLaterSeconds or else the node will fail to startup.
-             */
-            final int maxAttempts = 10;
-            ConnectionResponse response = null;
-            for (int i = 0; i < maxAttempts || retryIndefinitely; i++) {
-                try {
-                    response = senderListener.requestConnection(requestMsg).getConnectionResponse();
-                    if (response.isBlockedByFirewall()) {
-                        logger.warn("Connection request was blocked by cluster manager's firewall.");
-                        // set response to null and treat a firewall blockage the same as getting no response from manager
-                        response = null;
-                        break;
-                    } else if (response.shouldTryLater()) {
-                        logger.info("Flow controller requested by cluster manager to retry connection in " + response.getTryLaterSeconds() + " seconds.");
-                        try {
-                            Thread.sleep(response.getTryLaterSeconds() * 1000);
-                        } catch (final InterruptedException ie) {
-                            // we were interrupted, so finish quickly
-                            break;
-                        }
-                    } else {
-                        // we received a successful connection response from manager
-                        break;
-                    }
-
-                } catch (final Exception pe) {
-                    // could not create a socket and communicate with manager
-                    logger.warn("Failed to connect to cluster due to: " + pe, pe);
-                    if (retryOnCommsFailure) {
-                        try {
-                            Thread.sleep(connectionRetryMillis);
-                        } catch (final InterruptedException ie) {
-                            break;
-                        }
-                    } else {
-                        break;
-                    }
-                }
-            }
-
-            if (response == null) {
-                // if response is null, then either we had IO problems or we were blocked by firewall or we couldn't determine manager's address
-                return response;
-            } else if (response.shouldTryLater()) {
-                // if response indicates we should try later, then manager was unable to service our request. Just load local flow and move on.
-                return null;
-            } else {
-                // cluster manager provided a successful response with a current dataflow
-                return response;
-            }
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    private void loadFromConnectionResponse(final ConnectionResponse response) throws ConnectionException {
-        writeLock.lock();
-        try {
-
-            // get the dataflow from the response
-            final DataFlow dataFlow = response.getDataFlow();
-
-            // load new controller state
-            loadFromBytes(dataFlow, true);
-
-            // set node ID on controller before we start heartbeating because heartbeat needs node ID
-            nodeId = response.getNodeIdentifier();
-            logger.info("Setting Flow Controller's Node ID: " + nodeId);
-            controller.setNodeId(nodeId);
-
-            // mark the node as clustered
-            controller.setClustered(true, response.getInstanceId(), response.getClusterManagerDN());
-            controller.setClusterManagerRemoteSiteInfo(response.getManagerRemoteInputPort(), response.isManagerRemoteCommsSecure());
-
-            controller.setConnected(true);
-
-            // set primary
-            controller.setPrimary(response.isPrimary());
-
-            // start the processors as indicated by the dataflow
-            if (dataFlow.isAutoStartProcessors()) {
-                controller.startDelayed();
-            }
-
-            loadTemplates(dataFlow.getTemplates());
-            loadSnippets(dataFlow.getSnippets());
-            controller.startHeartbeating();
-        } catch (final UninheritableFlowException ufe) {
-            throw new ConnectionException(CONNECTION_EXCEPTION_MSG_PREFIX + "local flow is different than cluster flow.", ufe);
-        } catch (final FlowSerializationException fse) {
-            throw new ConnectionException(CONNECTION_EXCEPTION_MSG_PREFIX + "local or cluster flow is malformed.", fse);
-        } catch (final FlowSynchronizationException fse) {
-            throw new ConnectionException(CONNECTION_EXCEPTION_MSG_PREFIX + "local flow controller partially updated.  Administrator should disconnect node and review flow for corruption.", fse);
-        } catch (final Exception ex) {
-            throw new ConnectionException("Failed to connect node to cluster due to: " + ex, ex);
-        } finally {
-            writeLock.unlock();
-        }
-
-    }
-
-    @Override
-    public void copyCurrentFlow(final OutputStream os) throws IOException {
-        readLock.lock();
-        try {
-            if (!Files.exists(flowXml) || Files.size(flowXml) == 0) {
-                return;
-            }
-
-            try (final InputStream in = Files.newInputStream(flowXml, StandardOpenOption.READ);
-                    final InputStream gzipIn = new GZIPInputStream(in)) {
-                FileUtils.copy(gzipIn, os);
-            }
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    public void loadTemplates(final byte[] bytes) throws IOException {
-        if (bytes.length == 0) {
-            return;
-        }
-
-        controller.clearTemplates();
-
-        for (final Template template : TemplateManager.parseBytes(bytes)) {
-            controller.addTemplate(template.getDetails());
-        }
-    }
-
-    public void loadSnippets(final byte[] bytes) throws IOException {
-        if (bytes.length == 0) {
-            return;
-        }
-
-        final SnippetManager snippetManager = controller.getSnippetManager();
-        snippetManager.clear();
-
-        for (final StandardSnippet snippet : SnippetManager.parseBytes(bytes)) {
-            snippetManager.addSnippet(snippet);
-        }
-    }
-
-    @Override
-    public FlowController getController() {
-        return controller;
-    }
-
-    private class SaveReportingTask implements Runnable {
-
-        @Override
-        public void run() {
-            try {
-                //Hang onto the SaveHolder here rather than setting it to null because if the save fails we will try again
-                final SaveHolder holder = StandardFlowService.this.saveHolder.get();
-                if (holder == null) {
-                    return;
-                }
-
-                if (logger.isTraceEnabled()) {
-                    logger.trace("Save request time {} // Current time {}", holder.saveTime.getTime(), new Date());
-                }
-
-                final Calendar now = Calendar.getInstance();
-                if (holder.saveTime.before(now) || holder.shouldArchive) {
-                    if (logger.isTraceEnabled()) {
-                        logger.trace("Waiting for write lock and then will save");
-                    }
-                    writeLock.lock();
-                    try {
-                        dao.save(controller, holder.shouldArchive);
-                        // Nulling it out if it is still set to our current SaveHolder.  Otherwise leave it alone because it means 
-                        // another save is already pending.
-                        final boolean noSavePending = StandardFlowService.this.saveHolder.compareAndSet(holder, null);
-                        logger.info("Saved flow controller {} // Another save pending = {}", controller, !noSavePending);
-                    } finally {
-                        writeLock.unlock();
-                    }
-                }
-            } catch (final Throwable t) {
-                logger.error("Unable to save flow controller configuration due to: " + t, t);
-                if (logger.isDebugEnabled()) {
-                    logger.error("", t);
-                }
-
-                // record the failed save as a bulletin
-                final Bulletin saveFailureBulletin = BulletinFactory.createBulletin(EVENT_CATEGORY, LogLevel.ERROR.name(), "Unable to save flow controller configuration.");
-                controller.getBulletinRepository().addBulletin(saveFailureBulletin);
-            }
-        }
-    }
-
-    private class SaveHolder {
-
-        private final Calendar saveTime;
-        private final boolean shouldArchive;
-
-        private SaveHolder(final Calendar moment, final boolean archive) {
-            saveTime = moment;
-            shouldArchive = archive;
-        }
-    }
-
-    public boolean isPrimary() {
-        readLock.lock();
-        try {
-            return controller.isPrimary();
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    public void setPrimary(boolean primary) {
-        writeLock.lock();
-        try {
-            controller.setPrimary(primary);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-}


[12/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/node/Node.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/node/Node.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/node/Node.java
deleted file mode 100644
index 84565da..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/node/Node.java
+++ /dev/null
@@ -1,252 +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.cluster.node;
-
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.nifi.cluster.HeartbeatPayload;
-import org.apache.nifi.cluster.protocol.Heartbeat;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Represents a connected flow controller. Nodes always have an immutable
- * identifier and a status. The status may be changed, but never null.
- *
- * A Node may be cloned, but the cloning is a shallow copy of the instance.
- *
- * This class overrides hashCode and equals and considers two instances to be
- * equal if they have the equal NodeIdentifiers.
- *
- * @author unattributed
- */
-public class Node implements Cloneable, Comparable<Node> {
-
-    private static final Logger lockLogger = LoggerFactory.getLogger("cluster.lock");
-
-    /**
-     * The semantics of a Node status are as follows:
-     * <ul>
-     * <li>CONNECTED -- a flow controller that is connected to the cluster. A
-     * connecting node transitions to connected after the cluster receives the
-     * flow controller's first heartbeat. A connected node can transition to
-     * disconnecting.</li>
-     * <li>CONNECTING -- a flow controller has issued a connection request to
-     * the cluster, but has not yet sent a heartbeat. A connecting node can
-     * transition to disconnecting or connected. The cluster will not accept any
-     * external requests to change the flow while any node is connecting.</li>
-     * <li>DISCONNECTED -- a flow controller that is not connected to the
-     * cluster. A disconnected node can transition to connecting.</li>
-     * <li>DISCONNECTING -- a flow controller that is in the process of
-     * disconnecting from the cluster. A disconnecting node will always
-     * transition to disconnected.</li>
-     * </ul>
-     */
-    public static enum Status {
-
-        CONNECTED,
-        CONNECTING,
-        DISCONNECTED,
-        DISCONNECTING
-    }
-
-    /**
-     * the node's unique identifier
-     */
-    private final NodeIdentifier nodeId;
-
-    /**
-     * the node statue
-     */
-    private Status status;
-
-    /**
-     * the last heartbeat received by from the node
-     */
-    private Heartbeat lastHeartbeat;
-
-    /**
-     * the payload of the last heartbeat received from the node
-     */
-    private HeartbeatPayload lastHeartbeatPayload;
-
-    /**
-     * the last time the connection for this node was requested
-     */
-    private AtomicLong connectionRequestedTimestamp = new AtomicLong(0L);
-
-    /**
-     * a flag to indicate this node was disconnected because of a lack of
-     * heartbeat
-     */
-    private boolean heartbeatDisconnection;
-
-    public Node(final NodeIdentifier id, final Status status) {
-        if (id == null) {
-            throw new IllegalArgumentException("ID may not be null.");
-        } else if (status == null) {
-            throw new IllegalArgumentException("Status may not be null.");
-        }
-        this.nodeId = id;
-        this.status = status;
-    }
-
-    public NodeIdentifier getNodeId() {
-        return nodeId;
-    }
-
-    /**
-     * Returns the last received heartbeat or null if no heartbeat has been set.
-     *
-     * @return a heartbeat or null
-     */
-    public Heartbeat getHeartbeat() {
-        return lastHeartbeat;
-    }
-
-    public HeartbeatPayload getHeartbeatPayload() {
-        return lastHeartbeatPayload;
-    }
-
-    /**
-     * Sets the last heartbeat received.
-     *
-     * @param heartbeat a heartbeat
-     *
-     * @throws ProtocolException if the heartbeat's payload failed unmarshalling
-     */
-    public void setHeartbeat(final Heartbeat heartbeat) throws ProtocolException {
-        this.lastHeartbeat = heartbeat;
-        if (this.lastHeartbeat == null) {
-            this.lastHeartbeatPayload = null;
-        } else {
-            final byte[] payload = lastHeartbeat.getPayload();
-            if (payload == null || payload.length == 0) {
-                this.lastHeartbeatPayload = null;
-            } else {
-                this.lastHeartbeatPayload = HeartbeatPayload.unmarshal(payload);
-            }
-        }
-    }
-
-    /**
-     * Returns the time of the last received connection request for this node.
-     *
-     * @return the time when the connection request for this node was received.
-     */
-    public long getConnectionRequestedTimestamp() {
-        return connectionRequestedTimestamp.get();
-    }
-
-    /**
-     * Sets the time when the connection request for this node was last
-     * received.
-     *
-     * This method is thread-safe and may be called without obtaining any lock.
-     *
-     * @param connectionRequestedTimestamp
-     */
-    public void setConnectionRequestedTimestamp(long connectionRequestedTimestamp) {
-        this.connectionRequestedTimestamp.set(connectionRequestedTimestamp);
-    }
-
-    /**
-     * Returns true if the node was disconnected due to lack of heartbeat; false
-     * otherwise.
-     *
-     * @return true if the node was disconnected due to lack of heartbeat; false
-     * otherwise.
-     */
-    public boolean isHeartbeatDisconnection() {
-        return heartbeatDisconnection;
-    }
-
-    /**
-     * Sets the status to disconnected and flags the node as being disconnected
-     * by lack of heartbeat.
-     */
-    public void setHeartbeatDisconnection() {
-        setStatus(Status.DISCONNECTED);
-        heartbeatDisconnection = true;
-    }
-
-    /**
-     * @return the status
-     */
-    public Status getStatus() {
-        return status;
-    }
-
-    /**
-     * @param status a status
-     */
-    public void setStatus(final Status status) {
-        if (status == null) {
-            throw new IllegalArgumentException("Status may not be null.");
-        }
-        this.status = status;
-        heartbeatDisconnection = false;
-    }
-
-    @Override
-    public Node clone() {
-        final Node clone = new Node(nodeId, status);
-        clone.lastHeartbeat = lastHeartbeat;
-        clone.lastHeartbeatPayload = lastHeartbeatPayload;
-        clone.heartbeatDisconnection = heartbeatDisconnection;
-        clone.connectionRequestedTimestamp = connectionRequestedTimestamp;
-        return clone;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (getClass() != obj.getClass()) {
-            return false;
-        }
-        final Node other = (Node) obj;
-        if (this.nodeId != other.nodeId && (this.nodeId == null || !this.nodeId.equals(other.nodeId))) {
-            return false;
-        }
-        return true;
-    }
-
-    @Override
-    public int hashCode() {
-        int hash = 7;
-        hash = 53 * hash + (this.nodeId != null ? this.nodeId.hashCode() : 0);
-        return hash;
-    }
-
-    @Override
-    public String toString() {
-        return nodeId.toString();
-    }
-
-    @Override
-    public int compareTo(final Node o) {
-        if (o == null) {
-            return -1;
-        }
-        return getNodeId().getId().compareTo(o.getNodeId().getId());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java
deleted file mode 100644
index e26d196..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java
+++ /dev/null
@@ -1,116 +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.cluster.spring;
-
-import java.net.InetSocketAddress;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.cluster.protocol.impl.ClusterServiceDiscovery;
-import org.apache.nifi.cluster.protocol.impl.ClusterServiceLocator;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.NiFiProperties;
-
-import org.springframework.beans.BeansException;
-import org.springframework.beans.factory.DisposableBean;
-import org.springframework.beans.factory.FactoryBean;
-import org.springframework.context.ApplicationContext;
-import org.springframework.context.ApplicationContextAware;
-
-/**
- * Factory bean for creating a singleton ClusterManagerProtocolServiceLocator
- * instance. If the application is configured to act as the cluster manager,
- * then null is always returned as the created instance.
- *
- * The cluster manager protocol service represents the socket endpoint for
- * sending internal socket messages to the cluster manager.
- */
-public class ClusterManagerProtocolServiceLocatorFactoryBean implements FactoryBean, ApplicationContextAware, DisposableBean {
-
-    private ApplicationContext applicationContext;
-
-    private ClusterServiceLocator locator;
-
-    private NiFiProperties properties;
-
-    @Override
-    public Object getObject() throws Exception {
-        /*
-         * If configured for the cluster manager, then the service locator is never used.  
-         */
-        if (properties.isClusterManager()) {
-            return null;
-        } else if (locator == null) {
-
-            if (properties.getClusterProtocolUseMulticast()) {
-
-                // get the service discovery instance
-                final ClusterServiceDiscovery serviceDiscovery = applicationContext.getBean("clusterManagerProtocolServiceDiscovery", ClusterServiceDiscovery.class);
-
-                // create service location configuration
-                final ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig();
-                config.setNumAttempts(properties.getClusterProtocolMulticastServiceLocatorAttempts());
-
-                final int delay = (int) FormatUtils.getTimeDuration(properties.getClusterProtocolMulticastServiceLocatorAttemptsDelay(), TimeUnit.SECONDS);
-                config.setTimeBetweenAttempts(delay);
-                config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS);
-
-                locator = new ClusterServiceLocator(serviceDiscovery);
-                locator.setAttemptsConfig(config);
-
-            } else {
-                final String serviceName = applicationContext.getBean("clusterManagerProtocolServiceName", String.class);
-                final InetSocketAddress serviceAddress = properties.getClusterNodeUnicastManagerProtocolAddress();
-                final DiscoverableService service = new DiscoverableServiceImpl(serviceName, serviceAddress);
-                locator = new ClusterServiceLocator(service);
-            }
-
-            // start the locator
-            locator.start();
-
-        }
-        return locator;
-
-    }
-
-    @Override
-    public Class getObjectType() {
-        return ClusterServiceLocator.class;
-    }
-
-    @Override
-    public boolean isSingleton() {
-        return true;
-    }
-
-    @Override
-    public void setApplicationContext(ApplicationContext applicationContext) throws BeansException {
-        this.applicationContext = applicationContext;
-    }
-
-    @Override
-    public void destroy() throws Exception {
-        if (locator != null && locator.isRunning()) {
-            locator.stop();
-        }
-    }
-
-    public void setProperties(NiFiProperties properties) {
-        this.properties = properties;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java
deleted file mode 100644
index ef72298..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.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.cluster.spring;
-
-import java.io.File;
-import org.apache.nifi.cluster.firewall.impl.FileBasedClusterNodeFirewall;
-import org.apache.nifi.util.NiFiProperties;
-import org.springframework.beans.factory.FactoryBean;
-
-/**
- * Factory bean for creating a singleton FileBasedClusterNodeFirewall instance.
- */
-public class FileBasedClusterNodeFirewallFactoryBean implements FactoryBean {
-
-    private FileBasedClusterNodeFirewall firewall;
-
-    private NiFiProperties properties;
-
-    @Override
-    public Object getObject() throws Exception {
-        if (firewall == null) {
-            final File config = properties.getClusterManagerNodeFirewallFile();
-            final File restoreDirectory = properties.getRestoreDirectory();
-            if (config != null) {
-                firewall = new FileBasedClusterNodeFirewall(config, restoreDirectory);
-            }
-        }
-        return firewall;
-    }
-
-    @Override
-    public Class getObjectType() {
-        return FileBasedClusterNodeFirewall.class;
-    }
-
-    @Override
-    public boolean isSingleton() {
-        return true;
-    }
-
-    public void setProperties(NiFiProperties properties) {
-        this.properties = properties;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java
deleted file mode 100644
index 7169730..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java
+++ /dev/null
@@ -1,139 +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.cluster.spring;
-
-import java.nio.file.Paths;
-import org.apache.nifi.admin.service.AuditService;
-import org.apache.nifi.cluster.event.EventManager;
-import org.apache.nifi.cluster.firewall.ClusterNodeFirewall;
-import org.apache.nifi.cluster.flow.DataFlowManagementService;
-import org.apache.nifi.cluster.manager.HttpRequestReplicator;
-import org.apache.nifi.cluster.manager.HttpResponseMapper;
-import org.apache.nifi.cluster.manager.impl.WebClusterManager;
-import org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderListener;
-import org.apache.nifi.cluster.protocol.impl.ClusterServicesBroadcaster;
-import org.apache.nifi.controller.service.ControllerServiceLoader;
-import org.apache.nifi.encrypt.StringEncryptor;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-import org.apache.nifi.util.NiFiProperties;
-import org.springframework.beans.BeansException;
-import org.springframework.beans.factory.FactoryBean;
-import org.springframework.context.ApplicationContext;
-import org.springframework.context.ApplicationContextAware;
-
-/**
- * Factory bean for creating a singleton WebClusterManager instance. If the
- * application is not configured to act as the cluster manager, then null is
- * always returned as the created instance.
- */
-public class WebClusterManagerFactoryBean implements FactoryBean, ApplicationContextAware {
-
-    private ApplicationContext applicationContext;
-
-    private WebClusterManager clusterManager;
-
-    private NiFiProperties properties;
-
-    private StringEncryptor encryptor;
-
-    @Override
-    public Object getObject() throws Exception {
-        if (properties.isClusterManager() && properties.isNode()) {
-            throw new IllegalStateException("Application may be configured as a cluster manager or a node, but not both.");
-        } else if (!properties.isClusterManager()) {
-            /*
-             * If not configured for the cluster manager, then the cluster manager is never used.  
-             * null is returned so that we don't instantiate a thread pool or other resources. 
-             */
-            return null;
-        } else if (clusterManager == null) {
-
-            // get the service configuration path (fail early)
-            final String serviceConfigurationFile = properties.getProperty(NiFiProperties.SERVICE_CONFIGURATION_FILE);
-            if (serviceConfigurationFile == null) {
-                throw new NullPointerException("The service configuration file has not been specified.");
-            }
-
-            final HttpRequestReplicator requestReplicator = applicationContext.getBean("httpRequestReplicator", HttpRequestReplicator.class);
-            final HttpResponseMapper responseMapper = applicationContext.getBean("httpResponseMapper", HttpResponseMapper.class);
-            final DataFlowManagementService dataFlowService = applicationContext.getBean("dataFlowManagementService", DataFlowManagementService.class);
-            final ClusterManagerProtocolSenderListener senderListener = applicationContext.getBean("clusterManagerProtocolSenderListener", ClusterManagerProtocolSenderListener.class);
-
-            // create the manager
-            clusterManager = new WebClusterManager(
-                    requestReplicator,
-                    responseMapper,
-                    dataFlowService,
-                    senderListener,
-                    properties,
-                    encryptor
-            );
-
-            // set the service broadcaster
-            if (properties.getClusterProtocolUseMulticast()) {
-
-                // create broadcaster
-                final ClusterServicesBroadcaster broadcaster = applicationContext.getBean("clusterServicesBroadcaster", ClusterServicesBroadcaster.class);
-
-                // register the cluster manager protocol service
-                final String clusterManagerProtocolServiceName = applicationContext.getBean("clusterManagerProtocolServiceName", String.class);
-                final DiscoverableService clusterManagerProtocolService = new DiscoverableServiceImpl(clusterManagerProtocolServiceName, properties.getClusterManagerProtocolAddress());
-                broadcaster.addService(clusterManagerProtocolService);
-
-                clusterManager.setServicesBroadcaster(broadcaster);
-            }
-
-            // set the event manager
-            clusterManager.setEventManager(applicationContext.getBean("nodeEventHistoryManager", EventManager.class));
-
-            // set the cluster firewall
-            clusterManager.setClusterFirewall(applicationContext.getBean("clusterFirewall", ClusterNodeFirewall.class));
-
-            // set the audit service
-            clusterManager.setAuditService(applicationContext.getBean("auditService", AuditService.class));
-
-            // load the controller services
-            final ControllerServiceLoader serviceLoader = new ControllerServiceLoader(Paths.get(serviceConfigurationFile));
-            serviceLoader.loadControllerServices(clusterManager);
-        }
-        return clusterManager;
-    }
-
-    @Override
-    public Class getObjectType() {
-        return WebClusterManager.class;
-    }
-
-    @Override
-    public boolean isSingleton() {
-        return true;
-    }
-
-    @Override
-    public void setApplicationContext(ApplicationContext applicationContext) throws BeansException {
-        this.applicationContext = applicationContext;
-    }
-
-    public void setProperties(NiFiProperties properties) {
-        this.properties = properties;
-    }
-
-    public void setEncryptor(final StringEncryptor encryptor) {
-        this.encryptor = encryptor;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java
deleted file mode 100644
index 1ed5b30..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.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.controller.reporting;
-
-import org.apache.nifi.cluster.manager.impl.ClusteredReportingContext;
-import org.apache.nifi.controller.ProcessScheduler;
-import org.apache.nifi.controller.ValidationContextFactory;
-import org.apache.nifi.controller.service.ControllerServiceProvider;
-import org.apache.nifi.reporting.BulletinRepository;
-import org.apache.nifi.reporting.EventAccess;
-import org.apache.nifi.reporting.ReportingContext;
-import org.apache.nifi.reporting.ReportingTask;
-
-public class ClusteredReportingTaskNode extends AbstractReportingTaskNode {
-
-    private final EventAccess eventAccess;
-    private final BulletinRepository bulletinRepository;
-    private final ControllerServiceProvider serviceProvider;
-
-    public ClusteredReportingTaskNode(final ReportingTask reportingTask, final String id, final ProcessScheduler scheduler,
-            final EventAccess eventAccess, final BulletinRepository bulletinRepository, final ControllerServiceProvider serviceProvider,
-            final ValidationContextFactory validationContextFactory) {
-        super(reportingTask, id, serviceProvider, scheduler, validationContextFactory);
-
-        this.eventAccess = eventAccess;
-        this.bulletinRepository = bulletinRepository;
-        this.serviceProvider = serviceProvider;
-    }
-
-    @Override
-    public ReportingContext getReportingContext() {
-        return new ClusteredReportingContext(eventAccess, bulletinRepository, getProperties(), serviceProvider);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/main/resources/nifi-cluster-manager-context.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/resources/nifi-cluster-manager-context.xml b/nar-bundles/framework-bundle/framework/cluster/src/main/resources/nifi-cluster-manager-context.xml
deleted file mode 100644
index 68c29bc..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/main/resources/nifi-cluster-manager-context.xml
+++ /dev/null
@@ -1,124 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<!-- marked as lazy so that clustering beans are not created when applications runs in non-clustered mode -->
-<beans default-lazy-init="true"
-       xmlns="http://www.springframework.org/schema/beans"
-       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xmlns:util="http://www.springframework.org/schema/util"
-       xmlns:context="http://www.springframework.org/schema/context"
-       xmlns:aop="http://www.springframework.org/schema/aop"
-       xsi:schemaLocation="http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-3.1.xsd
-    http://www.springframework.org/schema/util http://www.springframework.org/schema/util/spring-util-3.1.xsd
-    http://www.springframework.org/schema/context http://www.springframework.org/schema/context/spring-context-3.1.xsd
-    http://www.springframework.org/schema/aop http://www.springframework.org/schema/aop/spring-aop-3.1.xsd">
-
-    <!-- jersey client -->
-    <bean id="jersey-client" class="org.apache.nifi.web.util.WebUtils" factory-method="createClient">
-        <constructor-arg>
-            <bean class="com.sun.jersey.api.client.config.DefaultClientConfig"/>
-        </constructor-arg>
-        <constructor-arg>
-            <bean class="org.apache.nifi.framework.security.util.SslContextFactory" factory-method="createSslContext">
-                <constructor-arg ref="nifiProperties"/>
-            </bean>
-        </constructor-arg>
-    </bean>
-
-    <!-- http request replicator -->
-    <bean id="httpRequestReplicator" class="org.apache.nifi.cluster.manager.impl.HttpRequestReplicatorImpl">
-        <constructor-arg index="0">
-            <bean factory-bean="nifiProperties" factory-method="getClusterManagerNodeApiRequestThreads"/>
-        </constructor-arg>
-        <constructor-arg ref="jersey-client" index="1"/>
-        <constructor-arg index="2">
-            <bean factory-bean="nifiProperties" factory-method="getClusterManagerNodeApiConnectionTimeout"/>
-        </constructor-arg>
-        <constructor-arg index="3">
-            <bean factory-bean="nifiProperties" factory-method="getClusterManagerNodeApiReadTimeout"/>
-        </constructor-arg>
-        <property name="nodeProtocolScheme">
-            <bean factory-bean="nifiProperties" factory-method="getClusterProtocolManagerToNodeApiScheme"/>
-        </property>
-    </bean>
-    
-    <!-- http response mapper -->
-    <bean id="httpResponseMapper" class="org.apache.nifi.cluster.manager.impl.HttpResponseMapperImpl"/>
-
-    <!-- cluster flow DAO -->
-    <bean id="dataFlowDao" class="org.apache.nifi.cluster.flow.impl.DataFlowDaoImpl">
-        <constructor-arg index="0">
-            <bean factory-bean="nifiProperties" factory-method="getFlowConfigurationFileDir"/>
-        </constructor-arg>
-        <constructor-arg index="1">
-            <bean factory-bean="nifiProperties" factory-method="getRestoreDirectory"/>
-        </constructor-arg>
-        <constructor-arg index="2">
-            <bean factory-bean="nifiProperties" factory-method="getAutoResumeState"/>
-        </constructor-arg>
-    </bean>
-    
-    <!-- dataflow management service -->
-    <bean id="dataFlowManagementService" class="org.apache.nifi.cluster.flow.impl.DataFlowManagementServiceImpl">
-        <constructor-arg ref="dataFlowDao"/>
-        <constructor-arg ref="clusterManagerProtocolSender"/>
-        <property name="retrievalDelay">
-            <bean factory-bean="nifiProperties" factory-method="getClusterManagerFlowRetrievalDelay"/>
-        </property>
-    </bean>
-
-    <!-- node event history manager -->
-    <bean id="nodeEventHistoryManager" class="org.apache.nifi.cluster.event.impl.EventManagerImpl">
-        <constructor-arg>
-            <bean factory-bean="nifiProperties" factory-method="getClusterManagerNodeEventHistorySize"/>
-        </constructor-arg>
-    </bean>
-
-    <!-- cluster firewall -->
-    <bean id="clusterFirewall" class="org.apache.nifi.cluster.spring.FileBasedClusterNodeFirewallFactoryBean">
-        <property name="properties" ref="nifiProperties"/>
-    </bean>
-
-    <!-- cluster manager -->
-    <bean id="clusterManager" class="org.apache.nifi.cluster.spring.WebClusterManagerFactoryBean">
-        <property name="properties" ref="nifiProperties"/>
-        <property name="encryptor" ref="stringEncryptor"/>
-    </bean>
-    
-    <!-- discoverable services -->
-    
-    <!-- cluster manager protocol discoverable service -->
-    
-    <!-- service name for communicating with the cluster manager using sockets -->
-    <bean id="clusterManagerProtocolServiceName" class="java.lang.String">
-        <constructor-arg value="cluster-manager-protocol" />
-    </bean>
-    
-    <!-- cluster manager protocol service discovery -->
-    <bean id="clusterManagerProtocolServiceDiscovery" class="org.apache.nifi.cluster.protocol.impl.ClusterServiceDiscovery">
-        <constructor-arg ref="clusterManagerProtocolServiceName" index="0"/>
-        <constructor-arg index="1">
-            <bean factory-bean="nifiProperties" factory-method="getClusterProtocolMulticastAddress"/>
-        </constructor-arg>
-        <constructor-arg ref="protocolMulticastConfiguration" index="2"/>
-        <constructor-arg ref="protocolContext" index="3"/>
-    </bean>
-    
-    <!-- cluster manager protocol service locator -->
-    <bean id="clusterManagerProtocolServiceLocator" class="org.apache.nifi.cluster.spring.ClusterManagerProtocolServiceLocatorFactoryBean">
-        <property name="properties" ref="nifiProperties"/>
-    </bean>
-        
-</beans>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java
deleted file mode 100644
index 09ea44b..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java
+++ /dev/null
@@ -1,119 +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.cluster.event.impl;
-
-import org.apache.nifi.cluster.event.impl.EventManagerImpl;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import org.apache.nifi.cluster.event.Event;
-import org.apache.nifi.cluster.event.Event.Category;
-import org.apache.nifi.cluster.event.EventManager;
-import org.junit.Test;
-import static org.junit.Assert.*;
-
-/**
- * @author unattributed
- */
-public class EventManagerImplTest {
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testNonPositiveHistorySize() {
-        new EventManagerImpl(0);
-    }
-
-    @Test
-    public void testGetEventsUnknownSource() {
-        EventManager manager = new EventManagerImpl(1);
-        assertEquals(Collections.EMPTY_LIST, manager.getEvents("unknown value"));
-    }
-
-    @Test
-    public void testGetEvents() {
-
-        EventManager manager = new EventManagerImpl(2);
-
-        Event e1 = new Event("1", "Event1", Category.INFO, 0);
-        Event e2 = new Event("1", "Event2", Category.INFO, 1);
-
-        manager.addEvent(e1);
-        manager.addEvent(e2);
-
-        List<Event> events = manager.getEvents("1");
-
-        // assert newest to oldest
-        assertEquals(Arrays.asList(e2, e1), events);
-    }
-
-    @Test
-    public void testGetMostRecentEventUnknownSource() {
-        EventManager manager = new EventManagerImpl(1);
-        assertNull(manager.getMostRecentEvent("unknown value"));
-    }
-
-    @Test
-    public void testGetMostRecentEvent() {
-
-        EventManager manager = new EventManagerImpl(2);
-
-        Event e1 = new Event("1", "Event1", Category.INFO, 0);
-        Event e2 = new Event("1", "Event2", Category.INFO, 1);
-
-        manager.addEvent(e1);
-        manager.addEvent(e2);
-
-        // assert newest to oldest
-        assertEquals(e2, manager.getMostRecentEvent("1"));
-    }
-
-    @Test
-    public void testAddEventExceedsHistorySize() {
-
-        EventManager manager = new EventManagerImpl(1);
-
-        Event e1 = new Event("1", "Event1", Category.INFO, 0);
-        Event e2 = new Event("1", "Event2", Category.INFO, 1);
-
-        manager.addEvent(e1);
-        manager.addEvent(e2);
-
-        List<Event> events = manager.getEvents("1");
-
-        // assert oldest evicted
-        assertEquals(Arrays.asList(e2), events);
-
-    }
-
-    @Test
-    public void testClearHistory() {
-
-        EventManager manager = new EventManagerImpl(1);
-
-        Event e1 = new Event("1", "Event1", Category.INFO, 0);
-        Event e2 = new Event("1", "Event2", Category.INFO, 1);
-
-        manager.addEvent(e1);
-        manager.addEvent(e2);
-
-        manager.clearEventHistory("1");
-
-        // assert oldest evicted
-        assertTrue(manager.getEvents("1").isEmpty());
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewallTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewallTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewallTest.java
deleted file mode 100644
index e5db7ca..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewallTest.java
+++ /dev/null
@@ -1,98 +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.cluster.firewall.impl;
-
-import java.io.File;
-import java.io.IOException;
-import org.apache.nifi.util.file.FileUtils;
-import org.junit.After;
-import org.junit.Before;
-import static org.junit.Assert.*;
-import org.junit.Test;
-
-public class FileBasedClusterNodeFirewallTest {
-
-    private FileBasedClusterNodeFirewall ipsFirewall;
-
-    private FileBasedClusterNodeFirewall acceptAllFirewall;
-
-    private File ipsConfig;
-
-    private File emptyConfig;
-
-    private File restoreDirectory;
-
-    @Before
-    public void setup() throws Exception {
-
-        ipsConfig = new File("src/test/resources/org/apache/nifi/cluster/firewall/impl/ips.txt");
-        emptyConfig = new File("src/test/resources/org/apache/nifi/cluster/firewall/impl/empty.txt");
-
-        restoreDirectory = new File(System.getProperty("java.io.tmpdir") + "/firewall_restore");
-
-        ipsFirewall = new FileBasedClusterNodeFirewall(ipsConfig, restoreDirectory);
-        acceptAllFirewall = new FileBasedClusterNodeFirewall(emptyConfig);
-    }
-
-    @After
-    public void teardown() throws IOException {
-        deleteFile(restoreDirectory);
-    }
-
-    @Test
-    public void testSyncWithRestore() {
-        assertEquals(ipsConfig.length(), new File(restoreDirectory, ipsConfig.getName()).length());
-    }
-
-    @Test
-    public void testIsPermissibleWithExactMatch() {
-        assertTrue(ipsFirewall.isPermissible("2.2.2.2"));
-    }
-
-    @Test
-    public void testIsPermissibleWithSubnetMatch() {
-        assertTrue(ipsFirewall.isPermissible("3.3.3.255"));
-    }
-
-    @Test
-    public void testIsPermissibleWithNoMatch() {
-        assertFalse(ipsFirewall.isPermissible("255.255.255.255"));
-    }
-
-    @Test
-    public void testIsPermissibleWithMalformedData() {
-        assertFalse(ipsFirewall.isPermissible("abc"));
-    }
-
-    @Test
-    public void testIsPermissibleWithEmptyConfig() {
-        assertTrue(acceptAllFirewall.isPermissible("1.1.1.1"));
-    }
-
-    @Test
-    public void testIsPermissibleWithEmptyConfigWithMalformedData() {
-        assertTrue(acceptAllFirewall.isPermissible("abc"));
-    }
-
-    private boolean deleteFile(final File file) {
-        if (file.isDirectory()) {
-            FileUtils.deleteFilesInDir(file, null, null, true, true);
-        }
-        return FileUtils.deleteFile(file, null, 10);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java
deleted file mode 100644
index f9ba016..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java
+++ /dev/null
@@ -1,343 +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.cluster.flow.impl;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Set;
-
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-
-import org.apache.nifi.cluster.flow.DataFlowDao;
-import org.apache.nifi.cluster.flow.PersistedFlowState;
-import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.StandardDataFlow;
-import org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderImpl;
-import org.apache.nifi.cluster.protocol.impl.SocketProtocolListener;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.io.socket.ServerSocketConfiguration;
-import org.apache.nifi.io.socket.SocketConfiguration;
-
-import org.apache.commons.io.FileUtils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.xml.sax.SAXException;
-
-/**
- * @author unattributed
- */
-public class DataFlowManagementServiceImplTest {
-
-    private DataFlowManagementServiceImpl service;
-    private File restoreLocation;
-    private File primaryLocation;
-    private DataFlowDao dao;
-    private int apiDummyPort;
-    private int socketPort;
-    private SocketConfiguration socketConfig;
-    private ClusterManagerProtocolSender sender;
-    private ServerSocketConfiguration serverSocketConfig;
-    private SocketProtocolListener listener;
-
-    @Before
-    public void setup() throws IOException {
-
-        primaryLocation = new File(System.getProperty("java.io.tmpdir") + "/primary" + this.getClass().getSimpleName());
-        restoreLocation = new File(System.getProperty("java.io.tmpdir") + "/restore" + this.getClass().getSimpleName());
-
-        FileUtils.deleteDirectory(primaryLocation);
-        FileUtils.deleteDirectory(restoreLocation);
-
-        ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
-
-        socketConfig = new SocketConfiguration();
-        socketConfig.setSocketTimeout(1000);
-        serverSocketConfig = new ServerSocketConfiguration();
-
-        dao = new DataFlowDaoImpl(primaryLocation, restoreLocation, false);
-
-        sender = new ClusterManagerProtocolSenderImpl(socketConfig, protocolContext);
-
-        service = new DataFlowManagementServiceImpl(dao, sender);
-        service.start();
-
-        listener = new SocketProtocolListener(1, 0, serverSocketConfig, protocolContext);
-        listener.start();
-
-        apiDummyPort = 7777;
-        socketPort = listener.getPort();
-    }
-
-    @After
-    public void teardown() throws IOException {
-
-        if (service != null && service.isRunning()) {
-            service.stop();
-        }
-
-        if (listener != null && listener.isRunning()) {
-            try {
-                listener.stop();
-            } catch (final Exception ex) {
-                ex.printStackTrace(System.out);
-            }
-        }
-        FileUtils.deleteDirectory(primaryLocation);
-        FileUtils.deleteDirectory(restoreLocation);
-
-    }
-
-    @Test
-    public void testLoadFlowWithNonExistentFlow() throws ParserConfigurationException, SAXException, IOException {
-        verifyFlow();
-    }
-
-    @Test
-    public void testLoadFlowWithNonExistentFlowWhenServiceStopped() throws IOException, SAXException, ParserConfigurationException {
-        service.stop();
-        verifyFlow();
-    }
-
-    private void verifyFlow() throws ParserConfigurationException, SAXException, IOException {
-        final byte[] flowBytes = service.loadDataFlow().getDataFlow().getFlow();
-        final DocumentBuilder docBuilder = DocumentBuilderFactory.newInstance().newDocumentBuilder();
-        final Document doc = docBuilder.parse(new ByteArrayInputStream(flowBytes));
-        final Element controller = (Element) doc.getElementsByTagName("flowController").item(0);
-        final Element rootGroup = (Element) controller.getElementsByTagName("rootGroup").item(0);
-        final String rootGroupName = rootGroup.getElementsByTagName("name").item(0).getTextContent();
-        assertEquals("NiFi Flow", rootGroupName);
-    }
-
-    @Test
-    public void testLoadFlowSingleNode() throws Exception {
-        String flowStr = "<rootGroup />";
-        byte[] flowBytes = flowStr.getBytes();
-        listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0])));
-
-        NodeIdentifier nodeId = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort);
-        service.setNodeIds(new HashSet<>(Arrays.asList(nodeId)));
-        service.setPersistedFlowState(PersistedFlowState.STALE);
-
-        assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState());
-
-        // sleep long enough for the flow retriever to run
-        waitForState(PersistedFlowState.CURRENT);
-
-        assertEquals(PersistedFlowState.CURRENT, service.getPersistedFlowState());
-        assertArrayEquals(flowBytes, service.loadDataFlow().getDataFlow().getFlow());
-
-    }
-
-    @Test
-    public void testLoadFlowWithSameNodeIds() throws Exception {
-
-        String flowStr = "<rootGroup />";
-        listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0])));
-
-        NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort);
-        NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort);
-        service.setNodeIds(new HashSet<>(Arrays.asList(nodeId1, nodeId2)));
-        service.setPersistedFlowState(PersistedFlowState.STALE);
-
-        assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState());
-
-        // sleep long enough for the flow retriever to run
-        waitForState(PersistedFlowState.CURRENT);
-
-        // verify that flow is current
-        assertEquals(PersistedFlowState.CURRENT, service.getPersistedFlowState());
-
-        // add same ids in different order
-        service.setNodeIds(new HashSet<>(Arrays.asList(nodeId2, nodeId1)));
-
-        // verify flow is still current
-        assertEquals(PersistedFlowState.CURRENT, service.getPersistedFlowState());
-
-    }
-
-    @Test
-    public void testLoadFlowWithABadNode() throws Exception {
-
-        String flowStr = "<rootGroup />";
-        byte[] flowBytes = flowStr.getBytes();
-        listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0])));
-
-        NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1);
-        NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort);
-        service.setNodeIds(new HashSet<>(Arrays.asList(nodeId1, nodeId2)));
-        service.setPersistedFlowState(PersistedFlowState.STALE);
-
-        assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState());
-
-        // sleep long enough for the flow retriever to run
-        waitForState(PersistedFlowState.CURRENT);
-
-        assertEquals(PersistedFlowState.CURRENT, service.getPersistedFlowState());
-        assertArrayEquals(flowBytes, service.loadDataFlow().getDataFlow().getFlow());
-
-    }
-
-    @Test
-    public void testLoadFlowWithConstantNodeIdChanging() throws Exception {
-        String flowStr = "<rootGroup />";
-        byte[] flowBytes = flowStr.getBytes();
-        listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0])));
-
-        NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1);
-        NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort);
-
-        for (int i = 0; i < 1000; i++) {
-            service.setNodeIds(new HashSet<>(Arrays.asList(nodeId1, nodeId2)));
-            service.setPersistedFlowState(PersistedFlowState.STALE);
-            assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState());
-        }
-
-        // sleep long enough for the flow retriever to run
-        waitForState(PersistedFlowState.CURRENT);
-
-        assertEquals(PersistedFlowState.CURRENT, service.getPersistedFlowState());
-        assertArrayEquals(flowBytes, service.loadDataFlow().getDataFlow().getFlow());
-    }
-
-    @Test
-    public void testLoadFlowWithConstantNodeIdChangingWithRetrievalDelay() throws Exception {
-
-        String flowStr = "<rootGroup />";
-        listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0])));
-
-        NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1);
-        NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort);
-
-        service.setRetrievalDelay("5 sec");
-        for (int i = 0; i < 1000; i++) {
-            service.setNodeIds(new HashSet<>(Arrays.asList(nodeId1, nodeId2)));
-            service.setPersistedFlowState(PersistedFlowState.STALE);
-            assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState());
-        }
-
-        // sleep long enough for the flow retriever to run
-        waitForState(PersistedFlowState.STALE);
-
-        assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState());
-
-    }
-
-    @Test
-    public void testStopRequestedWhileRetrieving() throws Exception {
-
-        String flowStr = "<rootGroup />";
-        listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0])));
-        Set<NodeIdentifier> nodeIds = new HashSet<>();
-        for (int i = 0; i < 1000; i++) {
-            nodeIds.add(new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1));
-        }
-        nodeIds.add(new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort));
-
-        long lastRetrievalTime = service.getLastRetrievalTime();
-
-        service.setNodeIds(nodeIds);
-        service.setPersistedFlowState(PersistedFlowState.STALE);
-        assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState());
-
-        // sleep long enough for the flow retriever to run
-        waitForState(PersistedFlowState.STALE);
-
-        service.stop();
-
-        service.setPersistedFlowState(PersistedFlowState.STALE);
-        assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState());
-
-        assertEquals(lastRetrievalTime, service.getLastRetrievalTime());
-
-    }
-
-    @Test
-    public void testLoadFlowUnknownState() throws Exception {
-
-        String flowStr = "<rootGroup />";
-        byte[] flowBytes = flowStr.getBytes();
-        listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0])));
-        NodeIdentifier nodeId = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort);
-
-        service.setNodeIds(new HashSet<>(Arrays.asList(nodeId)));
-        service.setPersistedFlowState(PersistedFlowState.STALE);
-        assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState());
-
-        service.setPersistedFlowState(PersistedFlowState.UNKNOWN);
-
-        assertEquals(PersistedFlowState.UNKNOWN, service.getPersistedFlowState());
-
-        service.setPersistedFlowState(PersistedFlowState.STALE);
-        assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState());
-
-        // sleep long enough for the flow retriever to run
-        waitForState(PersistedFlowState.CURRENT);
-
-        assertArrayEquals(flowBytes, service.loadDataFlow().getDataFlow().getFlow());
-
-    }
-
-    private class FlowRequestProtocolHandler implements ProtocolHandler {
-
-        private StandardDataFlow dataFlow;
-
-        public FlowRequestProtocolHandler(final StandardDataFlow dataFlow) {
-            this.dataFlow = dataFlow;
-        }
-
-        @Override
-        public boolean canHandle(ProtocolMessage msg) {
-            return true;
-        }
-
-        @Override
-        public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
-            FlowResponseMessage response = new FlowResponseMessage();
-            response.setDataFlow(dataFlow);
-            return response;
-        }
-
-    }
-
-    private void waitForState(PersistedFlowState state) throws InterruptedException {
-        for (int i = 0; i < 30; i++) {
-            if (service.getPersistedFlowState() == state) {
-                break;
-            } else {
-                Thread.sleep(1000);
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java
deleted file mode 100644
index 0c65aba..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java
+++ /dev/null
@@ -1,368 +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.cluster.manager.impl;
-
-import org.apache.nifi.cluster.manager.impl.HttpRequestReplicatorImpl;
-import javax.ws.rs.core.Response;
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.ws.rs.HttpMethod;
-import javax.ws.rs.core.MultivaluedMap;
-import java.net.URISyntaxException;
-import java.util.Arrays;
-import java.util.Iterator;
-import javax.ws.rs.core.StreamingOutput;
-import org.apache.nifi.cluster.manager.testutils.HttpResponse;
-import org.apache.nifi.cluster.manager.testutils.HttpServer;
-import com.sun.jersey.api.client.Client;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.net.URI;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import javax.ws.rs.core.Response.Status;
-import org.apache.nifi.cluster.manager.NodeResponse;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.apache.nifi.cluster.manager.testutils.HttpResponseAction;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import static org.junit.Assert.*;
-
-/**
- * @author unattributed
- */
-public class HttpRequestReplicatorImplTest {
-
-    private Client client;
-    private HttpRequestReplicatorImpl replicator;
-    private int executorThreadCount;
-    private int serverThreadCount;
-    private int serverPort;
-    private HttpServer server;
-    private Map<String, List<String>> expectedRequestParameters;
-    private Map<String, String> expectedRequestHeaders;
-    private Map<String, String> expectedResponseHeaders;
-    private Object expectedEntity;
-    private String expectedBody;
-    private URI prototypeUri;
-
-    @Before
-    public void setUp() throws IOException, URISyntaxException {
-
-        executorThreadCount = 5;
-        serverThreadCount = 3;
-
-        client = Client.create();
-
-        replicator = new HttpRequestReplicatorImpl(executorThreadCount, client, "1 sec", "1 sec");
-        replicator.start();
-
-        expectedRequestHeaders = new HashMap<>();
-        expectedRequestHeaders.put("header1", "header value1");
-        expectedRequestHeaders.put("header2", "header value2");
-
-        expectedRequestParameters = new HashMap<>();
-        expectedRequestParameters.put("param1", Arrays.asList("p value1"));
-        expectedRequestParameters.put("param2", Arrays.asList("p value2"));
-
-        expectedResponseHeaders = new HashMap<>();
-        expectedResponseHeaders.put("header1", "header value1");
-        expectedResponseHeaders.put("header2", "header value2");
-
-        expectedEntity = new Entity();
-
-        expectedBody = "some text";
-
-        prototypeUri = new URI("http://prototype.host/path/to/resource");
-
-        server = new HttpServer(serverThreadCount, 0);
-        server.start();
-        serverPort = server.getPort();
-    }
-
-    @After
-    public void teardown() {
-        if (server.isRunning()) {
-            server.stop();
-        }
-        if (replicator.isRunning()) {
-            replicator.stop();
-        }
-    }
-
-    @Test
-    public void testReplicateGetLessNodesThanReplicatorThreads() throws Throwable {
-        testReplicateXXX(executorThreadCount - 1, HttpMethod.GET);
-    }
-
-    @Test
-    public void testReplicateGetMoreNodesThanReplicatorThreads() throws Throwable {
-        testReplicateXXX(executorThreadCount + 1, HttpMethod.GET);
-    }
-
-    @Test
-    public void testReplicateGetWithUnresponsiveNode() throws Throwable {
-
-        // nodes
-        Set<NodeIdentifier> nodeIds = createNodes(2, "localhost", serverPort);
-
-        // response
-        HttpResponse expectedResponse = new HttpResponse(Status.OK, expectedBody);
-
-        // first response normal, second response slow
-        server.addResponseAction(new HttpResponseAction(expectedResponse));
-        server.addResponseAction(new HttpResponseAction(expectedResponse, 3500));
-
-        Set<NodeResponse> responses = replicator.replicate(
-                nodeIds,
-                HttpMethod.GET,
-                prototypeUri,
-                expectedRequestParameters,
-                expectedRequestHeaders);
-
-        assertEquals(nodeIds.size(), responses.size());
-
-        Iterator<NodeResponse> nodeResponseItr = responses.iterator();
-
-        NodeResponse firstResponse = nodeResponseItr.next();
-        NodeResponse secondResponse = nodeResponseItr.next();
-        NodeResponse goodResponse;
-        NodeResponse badResponse;
-        if (firstResponse.hasThrowable()) {
-            goodResponse = secondResponse;
-            badResponse = firstResponse;
-        } else {
-            goodResponse = firstResponse;
-            badResponse = secondResponse;
-        }
-
-        // good response
-        // check status
-        assertEquals(Status.OK.getStatusCode(), goodResponse.getStatus());
-
-        // check entity stream
-        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        ((StreamingOutput) goodResponse.getResponse().getEntity()).write(baos);
-        assertEquals("some text", new String(baos.toByteArray()));
-
-        // bad response
-        assertTrue(badResponse.hasThrowable());
-        assertEquals(Status.INTERNAL_SERVER_ERROR.getStatusCode(), badResponse.getStatus());
-
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testReplicateGetWithEntity() throws Throwable {
-        testReplicateXXXEntity(HttpMethod.GET);
-    }
-
-    @Test
-    public void testReplicatePost() throws Throwable {
-        testReplicateXXX(HttpMethod.POST);
-    }
-
-    @Test
-    public void testReplicatePostWithEntity() throws Throwable {
-        testReplicateXXXEntity(HttpMethod.POST);
-    }
-
-    @Test
-    public void testReplicatePut() throws Throwable {
-        testReplicateXXX(HttpMethod.PUT);
-    }
-
-    @Test
-    public void testReplicatePutWithEntity() throws Throwable {
-        testReplicateXXXEntity(HttpMethod.PUT);
-    }
-
-    @Test
-    public void testReplicateDelete() throws Throwable {
-        testReplicateXXX(HttpMethod.DELETE);
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testReplicateDeleteWithEntity() throws Throwable {
-        testReplicateXXXEntity(HttpMethod.DELETE);
-    }
-
-    @Test
-    public void testReplicateHead() throws Throwable {
-        testReplicateXXX(HttpMethod.HEAD);
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testReplicateHeadWithEntity() throws Throwable {
-        testReplicateXXXEntity(HttpMethod.HEAD);
-    }
-
-    @Test
-    public void testReplicateOptions() throws Throwable {
-        testReplicateXXX(HttpMethod.OPTIONS);
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testReplicateOptionsWithEntity() throws Throwable {
-        testReplicateXXXEntity(HttpMethod.OPTIONS);
-    }
-
-    private void testReplicateXXX(final String method) throws Throwable {
-        testReplicateXXX(executorThreadCount, method);
-    }
-
-    private void testReplicateXXX(final int numNodes, final String method) throws Throwable {
-
-        // nodes
-        Set<NodeIdentifier> nodeIds = createNodes(numNodes, "localhost", serverPort);
-
-        // set up responses
-        for (int i = 0; i < nodeIds.size(); i++) {
-            HttpResponse response = new HttpResponse(Status.OK, expectedBody);
-            response.addHeaders(expectedResponseHeaders);
-            server.addResponseAction(new HttpResponseAction(response));
-        }
-
-        // setup request parameters
-        server.addCheckedParameters(expectedRequestParameters);
-
-        // request headers
-        server.addCheckedHeaders(expectedRequestHeaders);
-
-        Set<NodeResponse> responses = replicator.replicate(
-                nodeIds,
-                method,
-                prototypeUri,
-                expectedRequestParameters,
-                expectedRequestHeaders);
-
-        Set<NodeIdentifier> returnedNodeIds = new HashSet<>();
-        for (NodeResponse response : responses) {
-
-            // check if we received an exception
-            if (response.hasThrowable()) {
-                throw response.getThrowable();
-            }
-
-            // gather ids to verify later
-            returnedNodeIds.add(response.getNodeId());
-
-            // check status
-            assertEquals(Status.OK.getStatusCode(), response.getStatus());
-
-            Response serverResponse = response.getResponse();
-
-            // check response headers are copied
-            assertTrue(containsHeaders(expectedResponseHeaders, serverResponse.getMetadata()));
-
-            // check entity stream
-            if (HttpMethod.HEAD.equalsIgnoreCase(method)) {
-                assertNull(serverResponse.getEntity());
-            } else {
-                assertTrue(isEquals((StreamingOutput) serverResponse.getEntity(), expectedBody));
-            }
-
-        }
-
-        // check node Ids
-        assertEquals(nodeIds, returnedNodeIds);
-    }
-
-    private void testReplicateXXXEntity(final String method) throws Throwable {
-        testReplicateXXXEntity(executorThreadCount, method);
-    }
-
-    private void testReplicateXXXEntity(final int numNodes, final String method) throws Throwable {
-
-        // nodes
-        Set<NodeIdentifier> nodeIds = createNodes(numNodes, "localhost", serverPort);
-
-        // set up responses
-        for (int i = 0; i < nodeIds.size(); i++) {
-            HttpResponse response = new HttpResponse(Status.OK, expectedBody);
-            response.addHeaders(expectedResponseHeaders);
-            server.addResponseAction(new HttpResponseAction(response));
-        }
-
-        // headers
-        expectedRequestHeaders.put("Content-Type", "application/xml");
-
-        Set<NodeResponse> responses = replicator.replicate(
-                nodeIds,
-                method,
-                prototypeUri,
-                expectedEntity,
-                expectedRequestHeaders);
-
-        Set<NodeIdentifier> returnedNodeIds = new HashSet<>();
-        for (NodeResponse response : responses) {
-
-            // check if we received an exception
-            if (response.hasThrowable()) {
-                throw response.getThrowable();
-            }
-
-            // gather ids to verify later
-            returnedNodeIds.add(response.getNodeId());
-
-            // check status
-            assertEquals(Status.OK.getStatusCode(), response.getStatus());
-
-            Response serverResponse = response.getResponse();
-
-            // check response headers are copied
-            assertTrue(containsHeaders(expectedResponseHeaders, serverResponse.getMetadata()));
-
-            // check entity stream
-            assertTrue(isEquals((StreamingOutput) serverResponse.getEntity(), expectedBody));
-
-        }
-
-        // check node Ids
-        assertEquals(nodeIds, returnedNodeIds);
-    }
-
-    private Set<NodeIdentifier> createNodes(int num, String host, int apiPort) {
-        Set<NodeIdentifier> result = new HashSet<>();
-        for (int i = 0; i < num; i++) {
-            result.add(new NodeIdentifier(String.valueOf(i), host, apiPort, host, 1));
-        }
-        return result;
-    }
-
-    private boolean isEquals(StreamingOutput so, String expectedText) throws IOException {
-        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        so.write(baos);
-        return expectedText.equals(new String(baos.toByteArray()));
-    }
-
-    private boolean containsHeaders(Map<String, String> expectedHeaders, MultivaluedMap<String, Object> metadata) {
-        for (Map.Entry<String, String> expectedEntry : expectedHeaders.entrySet()) {
-            if (expectedEntry.getValue().equals(metadata.getFirst(expectedEntry.getKey())) == false) {
-                return false;
-            }
-        }
-        return true;
-    }
-
-}
-
-@XmlRootElement
-class Entity {
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java
deleted file mode 100644
index d45a4d1..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java
+++ /dev/null
@@ -1,126 +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.cluster.manager.impl;
-
-import org.apache.nifi.cluster.manager.impl.HttpResponseMapperImpl;
-import com.sun.jersey.api.client.ClientResponse;
-import com.sun.jersey.core.util.MultivaluedMapImpl;
-import java.io.ByteArrayInputStream;
-import java.util.Map;
-import java.util.HashSet;
-import java.util.Set;
-import java.net.URI;
-import java.net.URISyntaxException;
-import org.apache.nifi.cluster.manager.NodeResponse;
-import org.apache.nifi.cluster.node.Node;
-import org.apache.nifi.cluster.node.Node.Status;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.junit.Before;
-import org.junit.Test;
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-
-/**
- * @author unattributed
- */
-public class HttpResponseMapperImplTest {
-
-    private HttpResponseMapperImpl mapper;
-
-    private URI dummyUri;
-
-    @Before
-    public void setup() throws URISyntaxException {
-        mapper = new HttpResponseMapperImpl();
-        dummyUri = new URI("http://dummy.com");
-    }
-
-    @Test
-    public void testToNodeStatusWithNo2xxResponses() {
-
-        Set<NodeResponse> nodeResponses = new HashSet<>();
-        nodeResponses.add(createNodeResourceResponse("1", 400));
-        nodeResponses.add(createNodeResourceResponse("2", 100));
-        nodeResponses.add(createNodeResourceResponse("3", 300));
-        nodeResponses.add(createNodeResourceResponse("4", 500));
-
-        Map<NodeResponse, Status> map = mapper.map(dummyUri, nodeResponses);
-
-        // since no 2xx responses, any 5xx is disconnected
-        for (Map.Entry<NodeResponse, Status> entry : map.entrySet()) {
-            NodeResponse response = entry.getKey();
-            Status status = entry.getValue();
-            switch (response.getNodeId().getId()) {
-                case "1":
-                    assertTrue(status == Node.Status.CONNECTED);
-                    break;
-                case "2":
-                    assertTrue(status == Node.Status.CONNECTED);
-                    break;
-                case "3":
-                    assertTrue(status == Node.Status.CONNECTED);
-                    break;
-                case "4":
-                    assertTrue(status == Node.Status.DISCONNECTED);
-                    break;
-            }
-        }
-    }
-
-    @Test
-    public void testToNodeStatusWith2xxResponses() {
-
-        Set<NodeResponse> nodeResponses = new HashSet<>();
-        nodeResponses.add(createNodeResourceResponse("1", 200));
-        nodeResponses.add(createNodeResourceResponse("2", 100));
-        nodeResponses.add(createNodeResourceResponse("3", 300));
-        nodeResponses.add(createNodeResourceResponse("4", 500));
-
-        Map<NodeResponse, Status> map = mapper.map(dummyUri, nodeResponses);
-
-        // since there were 2xx responses, any non-2xx is disconnected
-        for (Map.Entry<NodeResponse, Status> entry : map.entrySet()) {
-            NodeResponse response = entry.getKey();
-            Status status = entry.getValue();
-            switch (response.getNodeId().getId()) {
-                case "1":
-                    assertTrue(status == Node.Status.CONNECTED);
-                    break;
-                case "2":
-                    assertTrue(status == Node.Status.DISCONNECTED);
-                    break;
-                case "3":
-                    assertTrue(status == Node.Status.DISCONNECTED);
-                    break;
-                case "4":
-                    assertTrue(status == Node.Status.DISCONNECTED);
-                    break;
-            }
-        }
-    }
-
-    private NodeResponse createNodeResourceResponse(String nodeId, int statusCode) {
-
-        ClientResponse clientResponse = mock(ClientResponse.class);
-        when(clientResponse.getStatus()).thenReturn(statusCode);
-        when(clientResponse.getHeaders()).thenReturn(new MultivaluedMapImpl());
-        when(clientResponse.getEntityInputStream()).thenReturn(new ByteArrayInputStream(new byte[0]));
-
-        NodeIdentifier nodeIdentifier = new NodeIdentifier(nodeId, "localhost", 1, "localhost", 1);
-        return new NodeResponse(nodeIdentifier, "GET", dummyUri, clientResponse, 1L, "111");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/TestWebClusterManager.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/TestWebClusterManager.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/TestWebClusterManager.java
deleted file mode 100644
index 13a192f..0000000
--- a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/TestWebClusterManager.java
+++ /dev/null
@@ -1,54 +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.cluster.manager.impl;
-
-import org.apache.nifi.cluster.manager.impl.WebClusterManager;
-
-import static org.junit.Assert.assertEquals;
-
-import java.text.DateFormat;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Locale;
-
-import org.junit.Test;
-
-public class TestWebClusterManager {
-
-    @Test
-    public void testNormalizedStatusSnapshotDate() throws ParseException {
-        final DateFormat df = new SimpleDateFormat("yyyy/MM/dd HH:mm:SS.SSS", Locale.US);
-        final Date date1 = df.parse("2014/01/01 00:00:00.000");
-        final Date date2 = df.parse("2014/01/01 00:04:59.999");
-        final Date date3 = df.parse("2014/01/01 00:05:00.000");
-        final Date date4 = df.parse("2014/01/01 00:05:00.001");
-
-        final Date normalized1 = WebClusterManager.normalizeStatusSnapshotDate(date1, 300000);
-        assertEquals(date1, normalized1);
-
-        final Date normalized2 = WebClusterManager.normalizeStatusSnapshotDate(date2, 300000);
-        assertEquals(date1, normalized2);
-
-        final Date normalized3 = WebClusterManager.normalizeStatusSnapshotDate(date3, 300000);
-        assertEquals(date3, normalized3);
-
-        final Date normalized4 = WebClusterManager.normalizeStatusSnapshotDate(date4, 300000);
-        assertEquals(date3, normalized4);
-    }
-
-}


[20/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserEntity.java
deleted file mode 100644
index 963e853..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.UserDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a UserDTO.
- */
-@XmlRootElement(name = "userEntity")
-public class UserEntity extends Entity {
-
-    private UserDTO user;
-
-    /**
-     * The UserDTO that is being serialized.
-     *
-     * @return The UserDTO object
-     */
-    public UserDTO getUser() {
-        return user;
-    }
-
-    public void setUser(UserDTO user) {
-        this.user = user;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserGroupEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserGroupEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserGroupEntity.java
deleted file mode 100644
index a6542c8..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserGroupEntity.java
+++ /dev/null
@@ -1,45 +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.api.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.UserGroupDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a UserGroupDTO.
- */
-@XmlRootElement(name = "userGroupEntity")
-public class UserGroupEntity extends Entity {
-
-    private UserGroupDTO userGroup;
-
-    /**
-     * The UserGroupDTO that is being serialized.
-     *
-     * @return The UserGroupDTO object
-     */
-    public UserGroupDTO getUserGroup() {
-        return userGroup;
-    }
-
-    public void setUserGroup(UserGroupDTO userGroup) {
-        this.userGroup = userGroup;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserSearchResultsEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserSearchResultsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserSearchResultsEntity.java
deleted file mode 100644
index baffe15..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserSearchResultsEntity.java
+++ /dev/null
@@ -1,61 +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.api.entity;
-
-import java.util.List;
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.search.UserGroupSearchResultDTO;
-import org.apache.nifi.web.api.dto.search.UserSearchResultDTO;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to UserSearchResultDTOs and UserGroupSearchResultDTOs.
- */
-@XmlRootElement(name = "userSearchResultsEntity")
-public class UserSearchResultsEntity {
-
-    private List<UserSearchResultDTO> userResults;
-    private List<UserGroupSearchResultDTO> userGroupResults;
-
-    /**
-     * The user search results.
-     *
-     * @return
-     */
-    public List<UserSearchResultDTO> getUserResults() {
-        return userResults;
-    }
-
-    public void setUserResults(List<UserSearchResultDTO> userResults) {
-        this.userResults = userResults;
-    }
-
-    /**
-     * The user group search results.
-     *
-     * @return
-     */
-    public List<UserGroupSearchResultDTO> getUserGroupResults() {
-        return userGroupResults;
-    }
-
-    public void setUserGroupResults(List<UserGroupSearchResultDTO> userGroupResults) {
-        this.userGroupResults = userGroupResults;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UsersEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UsersEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UsersEntity.java
deleted file mode 100644
index 2d11d1f..0000000
--- a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UsersEntity.java
+++ /dev/null
@@ -1,64 +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.api.entity;
-
-import java.util.Collection;
-import java.util.Date;
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.UserDTO;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * A serialized representation of this class can be placed in the entity body of
- * a request or response to or from the API. This particular entity holds a
- * reference to a collection of UserDTO.
- */
-@XmlRootElement(name = "usersEntity")
-public class UsersEntity extends Entity {
-
-    private Collection<UserDTO> users;
-    private Date generated;
-
-    /**
-     * The collection of UserDTOs that are being serialized.
-     *
-     * @return The UserDTO object
-     */
-    public Collection<UserDTO> getUsers() {
-        return users;
-    }
-
-    public void setUsers(Collection<UserDTO> users) {
-        this.users = users;
-    }
-
-    /**
-     * When this content was generated.
-     *
-     * @return
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    public Date getGenerated() {
-        return generated;
-    }
-
-    public void setGenerated(Date generated) {
-        this.generated = generated;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-authorization-provider/.gitignore
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/.gitignore b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/.gitignore
deleted file mode 100755
index ea8c4bf..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/.gitignore
+++ /dev/null
@@ -1 +0,0 @@
-/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-authorization-provider/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/pom.xml b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/pom.xml
deleted file mode 100644
index b8960c3..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/pom.xml
+++ /dev/null
@@ -1,48 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-framework-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-
-    <artifactId>cluster-authorization-provider</artifactId>
-    <name>NiFi Framework Cluster Authority Provider</name>
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>file-authorization-provider</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>framework-cluster-protocol</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>framework-cluster</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-socket-utils</artifactId>
-        </dependency>
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/ClusterManagerAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/ClusterManagerAuthorizationProvider.java b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/ClusterManagerAuthorizationProvider.java
deleted file mode 100644
index 2b3b38c..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/ClusterManagerAuthorizationProvider.java
+++ /dev/null
@@ -1,225 +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.cluster.authorization;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.AuthorityProviderConfigurationContext;
-import org.apache.nifi.authorization.AuthorityProviderInitializationContext;
-import org.apache.nifi.authorization.FileAuthorizationProvider;
-import org.apache.nifi.authorization.annotation.AuthorityProviderContext;
-import org.apache.nifi.authorization.exception.ProviderCreationException;
-import org.apache.nifi.authorization.exception.ProviderDestructionException;
-import org.apache.nifi.cluster.authorization.protocol.message.DoesDnExistMessage;
-import org.apache.nifi.cluster.authorization.protocol.message.GetAuthoritiesMessage;
-import org.apache.nifi.cluster.authorization.protocol.message.GetGroupForUserMessage;
-import org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage;
-import static org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage.MessageType.DOES_DN_EXIST;
-import static org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage.MessageType.GET_AUTHORITIES;
-import static org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage.MessageType.GET_GROUP_FOR_USER;
-import org.apache.nifi.cluster.authorization.protocol.message.jaxb.JaxbProtocolUtils;
-import org.apache.nifi.cluster.manager.impl.WebClusterManager;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.io.socket.ServerSocketConfiguration;
-import org.apache.nifi.io.socket.SocketListener;
-import org.apache.nifi.io.socket.SocketUtils;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-import org.apache.nifi.logging.NiFiLog;
-import org.apache.nifi.util.NiFiProperties;
-import static org.apache.nifi.util.NiFiProperties.CLUSTER_MANAGER_ADDRESS;
-import org.apache.nifi.util.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.beans.BeansException;
-import org.springframework.context.ApplicationContext;
-import org.springframework.context.ApplicationContextAware;
-
-/**
- * Provides authorities for the NCM in clustered environments. Communication
- * occurs over TCP/IP sockets. All method calls are deferred to the
- * FileAuthorizationProvider.
- */
-public class ClusterManagerAuthorizationProvider extends FileAuthorizationProvider implements AuthorityProvider, ApplicationContextAware {
-
-    public static final String AUTHORITY_PROVIDER_SERVIVE_NAME = "cluster-authority-provider";
-
-    private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(ClusterManagerAuthorizationProvider.class));
-    private static final String CLUSTER_MANAGER_AUTHORITY_PROVIDER_PORT = "Authority Provider Port";
-    private static final String CLUSTER_MANAGER_AUTHORITY_PROVIDER_THREADS = "Authority Provider Threads";
-    private static final int DEFAULT_CLUSTER_MANAGER_AUTHORITY_PROVIDER_THREADS = 10;
-
-    private WebClusterManager clusterManager;
-    private ProtocolContext<ProtocolMessage> authorityProviderProtocolContext;
-    private SocketListener socketListener;
-    private NiFiProperties properties;
-    private ApplicationContext applicationContext;
-
-    @Override
-    public void initialize(final AuthorityProviderInitializationContext initializationContext) throws ProviderCreationException {
-        super.initialize(initializationContext);
-    }
-
-    @Override
-    public void onConfigured(final AuthorityProviderConfigurationContext configurationContext) throws ProviderCreationException {
-        super.onConfigured(configurationContext);
-
-        // get the socket address of the cluster authority provider
-        final InetSocketAddress clusterAuthorityProviderAddress = getClusterManagerAuthorityProviderAddress(configurationContext);
-
-        // get the cluster manager
-        clusterManager = applicationContext.getBean("clusterManager", WebClusterManager.class);
-
-        // if using multicast, then the authority provider's service is broadcasted
-        if (properties.getClusterProtocolUseMulticast()) {
-
-            // create the authority provider service for discovery
-            final DiscoverableService clusterAuthorityProviderService = new DiscoverableServiceImpl(AUTHORITY_PROVIDER_SERVIVE_NAME, clusterAuthorityProviderAddress);
-
-            // register the authority provider service with the cluster manager
-            clusterManager.addBroadcastedService(clusterAuthorityProviderService);
-        }
-
-        // get the number of protocol listening thread
-        final int numThreads = getClusterManagerAuthorityProviderThreads(configurationContext);
-
-        // the server socket configuration
-        final ServerSocketConfiguration configuration = applicationContext.getBean("protocolServerSocketConfiguration", ServerSocketConfiguration.class);
-
-        // the authority provider listens for node messages
-        socketListener = new SocketListener(numThreads, clusterAuthorityProviderAddress.getPort(), configuration) {
-            @Override
-            public void dispatchRequest(final Socket socket) {
-                ClusterManagerAuthorizationProvider.this.dispatchRequest(socket);
-            }
-        };
-
-        // start the socket listener
-        if (socketListener != null && !socketListener.isRunning()) {
-            try {
-                socketListener.start();
-            } catch (final IOException ioe) {
-                throw new ProviderCreationException("Failed to start Cluster Manager Authorization Provider due to: " + ioe, ioe);
-            }
-        }
-
-        // initialize the protocol context
-        authorityProviderProtocolContext = new JaxbProtocolContext<ProtocolMessage>(JaxbProtocolUtils.JAXB_CONTEXT);
-    }
-
-    @Override
-    public void preDestruction() throws ProviderDestructionException {
-        if (socketListener != null && socketListener.isRunning()) {
-            try {
-                socketListener.stop();
-            } catch (final IOException ioe) {
-                throw new ProviderDestructionException("Failed to stop Cluster Manager Authorization Provider due to: " + ioe, ioe);
-            }
-        }
-        super.preDestruction();
-    }
-
-    private int getClusterManagerAuthorityProviderThreads(final AuthorityProviderConfigurationContext configurationContext) {
-        try {
-            return Integer.parseInt(configurationContext.getProperty(CLUSTER_MANAGER_AUTHORITY_PROVIDER_THREADS));
-        } catch (NumberFormatException nfe) {
-            return DEFAULT_CLUSTER_MANAGER_AUTHORITY_PROVIDER_THREADS;
-        }
-    }
-
-    private InetSocketAddress getClusterManagerAuthorityProviderAddress(final AuthorityProviderConfigurationContext configurationContext) {
-        try {
-            String socketAddress = properties.getProperty(CLUSTER_MANAGER_ADDRESS);
-            if (StringUtils.isBlank(socketAddress)) {
-                socketAddress = "localhost";
-            }
-            return InetSocketAddress.createUnresolved(socketAddress, getClusterManagerAuthorityProviderPort(configurationContext));
-        } catch (Exception ex) {
-            throw new RuntimeException("Invalid manager authority provider address/port due to: " + ex, ex);
-        }
-    }
-
-    private Integer getClusterManagerAuthorityProviderPort(final AuthorityProviderConfigurationContext configurationContext) {
-        final String authorityProviderPort = configurationContext.getProperty(CLUSTER_MANAGER_AUTHORITY_PROVIDER_PORT);
-        if (authorityProviderPort == null || authorityProviderPort.trim().isEmpty()) {
-            throw new ProviderCreationException("The authority provider port must be specified.");
-        }
-
-        return Integer.parseInt(authorityProviderPort);
-    }
-
-    private void dispatchRequest(final Socket socket) {
-        try {
-            // unmarshall message
-            final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = authorityProviderProtocolContext.createUnmarshaller();
-            final ProtocolMessage request = unmarshaller.unmarshal(socket.getInputStream());
-            final ProtocolMessage response = request;
-
-            try {
-                switch (request.getType()) {
-                    case DOES_DN_EXIST: {
-                        final DoesDnExistMessage castedMsg = (DoesDnExistMessage) request;
-                        castedMsg.setResponse(doesDnExist(castedMsg.getDn()));
-                        break;
-                    }
-                    case GET_AUTHORITIES: {
-                        final GetAuthoritiesMessage castedMsg = (GetAuthoritiesMessage) request;
-                        castedMsg.setResponse(getAuthorities(castedMsg.getDn()));
-                        break;
-                    }
-                    case GET_GROUP_FOR_USER: {
-                        final GetGroupForUserMessage castedMsg = (GetGroupForUserMessage) request;
-                        castedMsg.setResponse(getGroupForUser(castedMsg.getDn()));
-                        break;
-                    }
-                    default: {
-                        throw new Exception("Unsupported Message Type: " + request.getType());
-                    }
-                }
-            } catch (final Exception ex) {
-                response.setExceptionClass(ex.getClass().getName());
-                response.setExceptionMessage(ex.getMessage());
-            }
-
-            final ProtocolMessageMarshaller<ProtocolMessage> marshaller = authorityProviderProtocolContext.createMarshaller();
-            marshaller.marshal(response, socket.getOutputStream());
-
-        } catch (final Exception e) {
-            logger.warn("Failed processing Socket Authorization Provider protocol message due to " + e, e);
-        } finally {
-            SocketUtils.closeQuietly(socket);
-        }
-    }
-
-    @Override
-    @AuthorityProviderContext
-    public void setApplicationContext(final ApplicationContext applicationContext) throws BeansException {
-        this.applicationContext = applicationContext;
-    }
-
-    @Override
-    @AuthorityProviderContext
-    public void setNiFiProperties(NiFiProperties properties) {
-        super.setNiFiProperties(properties);
-        this.properties = properties;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/NodeAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/NodeAuthorizationProvider.java b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/NodeAuthorizationProvider.java
deleted file mode 100644
index 840422f..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/NodeAuthorizationProvider.java
+++ /dev/null
@@ -1,389 +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.cluster.authorization;
-
-import org.apache.nifi.cluster.authorization.protocol.message.DoesDnExistMessage;
-import org.apache.nifi.cluster.authorization.protocol.message.GetAuthoritiesMessage;
-import org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import org.apache.nifi.authorization.Authority;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.AuthorityProviderConfigurationContext;
-import org.apache.nifi.authorization.AuthorityProviderInitializationContext;
-import org.apache.nifi.authorization.DownloadAuthorization;
-import org.apache.nifi.authorization.annotation.AuthorityProviderContext;
-import org.apache.nifi.authorization.exception.AuthorityAccessException;
-import org.apache.nifi.authorization.exception.IdentityAlreadyExistsException;
-import org.apache.nifi.authorization.exception.ProviderCreationException;
-import org.apache.nifi.authorization.exception.ProviderDestructionException;
-import org.apache.nifi.authorization.exception.UnknownIdentityException;
-import org.apache.nifi.cluster.authorization.protocol.message.GetGroupForUserMessage;
-import org.apache.nifi.cluster.authorization.protocol.message.jaxb.JaxbProtocolUtils;
-import org.apache.nifi.io.socket.SocketConfiguration;
-import org.apache.nifi.io.socket.SocketUtils;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller;
-import org.apache.nifi.cluster.protocol.impl.ClusterServiceDiscovery;
-import org.apache.nifi.cluster.protocol.impl.ClusterServiceLocator;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.apache.nifi.logging.NiFiLog;
-import org.apache.nifi.util.NiFiProperties;
-import static org.apache.nifi.util.NiFiProperties.CLUSTER_NODE_UNICAST_MANAGER_ADDRESS;
-import org.apache.nifi.util.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.beans.BeansException;
-import org.springframework.context.ApplicationContext;
-import org.springframework.context.ApplicationContextAware;
-
-/**
- * Provides authorities for nodes in clustered environments. Communication
- * occurs over TCP/IP sockets. All method calls are communicated to the cluster
- * manager provider via socket.
- */
-public class NodeAuthorizationProvider implements AuthorityProvider, ApplicationContextAware {
-
-    private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(NodeAuthorizationProvider.class));
-    private static final String CLUSTER_NODE_MANAGER_AUTHORITY_PROVIDER_PORT = "Cluster Manager Authority Provider Port";
-
-    private ProtocolContext<ProtocolMessage> authorityProviderProtocolContext;
-    private SocketConfiguration socketConfiguration;
-    private ClusterServiceLocator serviceLocator;
-    private ApplicationContext applicationContext;
-    private NiFiProperties properties;
-
-    @Override
-    public void initialize(AuthorityProviderInitializationContext initializationContext) throws ProviderCreationException {
-    }
-
-    @Override
-    public void onConfigured(final AuthorityProviderConfigurationContext configurationContext) throws ProviderCreationException {
-        // TODO clear user cache?
-
-        // if using multicast, then the authority provider's service is broadcasted
-        if (properties.getClusterProtocolUseMulticast()) {
-            // create the service discovery
-            final ClusterServiceDiscovery serviceDiscovery = new ClusterServiceDiscovery(
-                    ClusterManagerAuthorizationProvider.AUTHORITY_PROVIDER_SERVIVE_NAME,
-                    properties.getClusterProtocolMulticastAddress(),
-                    applicationContext.getBean("protocolMulticastConfiguration", MulticastConfiguration.class),
-                    applicationContext.getBean("protocolContext", ProtocolContext.class));
-
-            // create service location configuration
-            final ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig();
-            config.setNumAttempts(3);
-            config.setTimeBetweenAttempts(1);
-            config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS);
-
-            serviceLocator = new ClusterServiceLocator(serviceDiscovery);
-            serviceLocator.setAttemptsConfig(config);
-        } else {
-            final InetSocketAddress serviceAddress = getClusterNodeManagerAuthorityProviderAddress(configurationContext);
-            final DiscoverableService service = new DiscoverableServiceImpl(ClusterManagerAuthorizationProvider.AUTHORITY_PROVIDER_SERVIVE_NAME, serviceAddress);
-            serviceLocator = new ClusterServiceLocator(service);
-        }
-
-        try {
-            // start the service locator
-            serviceLocator.start();
-        } catch (final IOException ioe) {
-            throw new ProviderCreationException(ioe);
-        }
-
-        // the socket configuration
-        socketConfiguration = applicationContext.getBean("protocolSocketConfiguration", SocketConfiguration.class);
-
-        // initialize the protocol context
-        authorityProviderProtocolContext = new JaxbProtocolContext<ProtocolMessage>(JaxbProtocolUtils.JAXB_CONTEXT);
-    }
-
-    private InetSocketAddress getClusterNodeManagerAuthorityProviderAddress(final AuthorityProviderConfigurationContext configurationContext) {
-        try {
-            String socketAddress = properties.getProperty(CLUSTER_NODE_UNICAST_MANAGER_ADDRESS);
-            if (StringUtils.isBlank(socketAddress)) {
-                socketAddress = "localhost";
-            }
-            return InetSocketAddress.createUnresolved(socketAddress, getClusterNodeManagerAuthorityProviderPort(configurationContext));
-        } catch (Exception ex) {
-            throw new ProviderCreationException("Invalid cluster manager authority provider address/port due to: " + ex, ex);
-        }
-    }
-
-    private Integer getClusterNodeManagerAuthorityProviderPort(final AuthorityProviderConfigurationContext configurationContext) {
-        final String nodeAuthorityProviderPort = configurationContext.getProperty(CLUSTER_NODE_MANAGER_AUTHORITY_PROVIDER_PORT);
-        if (nodeAuthorityProviderPort == null || nodeAuthorityProviderPort.trim().isEmpty()) {
-            throw new ProviderCreationException("The cluster manager authority provider port must be specified.");
-        }
-
-        return Integer.parseInt(nodeAuthorityProviderPort);
-    }
-
-    @Override
-    public void setAuthorities(String dn, Set<Authority> authorities) throws AuthorityAccessException {
-        throw new AuthorityAccessException("Nodes are not allowed to set user authorities.");
-    }
-
-    @Override
-    public void addUser(String dn, String group) throws IdentityAlreadyExistsException, AuthorityAccessException {
-        throw new AuthorityAccessException("Nodes are not allowed to add users.");
-    }
-
-    @Override
-    public boolean doesDnExist(String dn) throws AuthorityAccessException {
-        // create message
-        final DoesDnExistMessage msg = new DoesDnExistMessage();
-        msg.setDn(dn);
-
-        Socket socket = null;
-        try {
-
-            final InetSocketAddress socketAddress = getServiceAddress();
-            if (socketAddress == null) {
-                throw new AuthorityAccessException("Cluster Authority Provider's address is not known.");
-            }
-
-            try {
-                // create a socket
-                socket = SocketUtils.createSocket(socketAddress, socketConfiguration);
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed to create socket due to: " + ioe, ioe);
-            }
-
-            try {
-                // marshal message to output stream
-                final ProtocolMessageMarshaller marshaller = authorityProviderProtocolContext.createMarshaller();
-                marshaller.marshal(msg, socket.getOutputStream());
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe);
-            }
-
-            try {
-
-                // unmarshall response and return
-                final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = authorityProviderProtocolContext.createUnmarshaller();
-                final DoesDnExistMessage response = (DoesDnExistMessage) unmarshaller.unmarshal(socket.getInputStream());
-
-                // check if there was an exception
-                if (response.wasException()) {
-                    throw new AuthorityAccessException(response.getExceptionMessage());
-                }
-
-                // return provider's response
-                return response.getResponse();
-
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed unmarshalling '" + msg.getType() + "' response protocol message due to: " + ioe, ioe);
-            }
-
-        } finally {
-            SocketUtils.closeQuietly(socket);
-        }
-    }
-
-    @Override
-    public Set<Authority> getAuthorities(String dn) throws UnknownIdentityException, AuthorityAccessException {
-        // create message
-        final GetAuthoritiesMessage msg = new GetAuthoritiesMessage();
-        msg.setDn(dn);
-
-        Socket socket = null;
-        try {
-
-            final InetSocketAddress socketAddress = getServiceAddress();
-            if (socketAddress == null) {
-                throw new AuthorityAccessException("Cluster Authority Provider's address is not known.");
-            }
-
-            try {
-                // create a socket
-                socket = SocketUtils.createSocket(socketAddress, socketConfiguration);
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed to create socket due to: " + ioe, ioe);
-            }
-
-            try {
-                // marshal message to output stream
-                final ProtocolMessageMarshaller marshaller = authorityProviderProtocolContext.createMarshaller();
-                marshaller.marshal(msg, socket.getOutputStream());
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe);
-            }
-
-            try {
-
-                // unmarshall response and return
-                final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = authorityProviderProtocolContext.createUnmarshaller();
-                final GetAuthoritiesMessage response = (GetAuthoritiesMessage) unmarshaller.unmarshal(socket.getInputStream());
-
-                // check if there was an exception
-                if (response.wasException()) {
-                    if (isException(UnknownIdentityException.class, response)) {
-                        throw new UnknownIdentityException(response.getExceptionMessage());
-                    } else {
-                        throw new AuthorityAccessException(response.getExceptionMessage());
-                    }
-                }
-
-                // return provider's response
-                return response.getResponse();
-
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed unmarshalling '" + msg.getType() + "' response protocol message due to: " + ioe, ioe);
-            }
-
-        } finally {
-            SocketUtils.closeQuietly(socket);
-        }
-    }
-
-    @Override
-    public Set<String> getUsers(Authority authority) throws AuthorityAccessException {
-        throw new AuthorityAccessException("Nodes are not allowed to get users for a given authority.");
-    }
-
-    @Override
-    public void revokeUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-        throw new AuthorityAccessException("Nodes are not allowed to revoke users.");
-    }
-
-    @Override
-    public void setUsersGroup(Set<String> dns, String group) throws UnknownIdentityException, AuthorityAccessException {
-        throw new AuthorityAccessException("Nodes are not allowed to set user groups.");
-    }
-
-    @Override
-    public void ungroupUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-        throw new AuthorityAccessException("Nodes are not allowed to ungroup users.");
-    }
-
-    @Override
-    public void ungroup(String group) throws AuthorityAccessException {
-        throw new AuthorityAccessException("Nodes are not allowed to ungroup.");
-    }
-
-    @Override
-    public DownloadAuthorization authorizeDownload(List<String> dnChain, Map<String, String> attributes) throws UnknownIdentityException, AuthorityAccessException {
-        return DownloadAuthorization.approved();
-    }
-
-    @Override
-    public String getGroupForUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-        // create message
-        final GetGroupForUserMessage msg = new GetGroupForUserMessage();
-        msg.setDn(dn);
-
-        Socket socket = null;
-        try {
-
-            final InetSocketAddress socketAddress = getServiceAddress();
-            if (socketAddress == null) {
-                throw new AuthorityAccessException("Cluster Authority Provider's address is not known.");
-            }
-
-            try {
-                // create a socket
-                socket = SocketUtils.createSocket(socketAddress, socketConfiguration);
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed to create socket due to: " + ioe, ioe);
-            }
-
-            try {
-                // marshal message to output stream
-                final ProtocolMessageMarshaller marshaller = authorityProviderProtocolContext.createMarshaller();
-                marshaller.marshal(msg, socket.getOutputStream());
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe);
-            }
-
-            try {
-
-                // unmarshall response and return
-                final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = authorityProviderProtocolContext.createUnmarshaller();
-                final GetGroupForUserMessage response = (GetGroupForUserMessage) unmarshaller.unmarshal(socket.getInputStream());
-
-                // check if there was an exception
-                if (response.wasException()) {
-                    if (isException(UnknownIdentityException.class, response)) {
-                        throw new UnknownIdentityException(response.getExceptionMessage());
-                    } else {
-                        throw new AuthorityAccessException(response.getExceptionMessage());
-                    }
-                }
-
-                return response.getResponse();
-            } catch (final IOException ioe) {
-                throw new AuthorityAccessException("Failed unmarshalling '" + msg.getType() + "' response protocol message due to: " + ioe, ioe);
-            }
-
-        } finally {
-            SocketUtils.closeQuietly(socket);
-        }
-    }
-
-    @Override
-    public void revokeGroup(String group) throws UnknownIdentityException, AuthorityAccessException {
-        throw new AuthorityAccessException("Nodes are not allowed to revoke groups.");
-    }
-
-    @Override
-    public void preDestruction() throws ProviderDestructionException {
-        try {
-            if (serviceLocator != null && serviceLocator.isRunning()) {
-                serviceLocator.stop();
-            }
-        } catch (final IOException ioe) {
-            throw new ProviderDestructionException(ioe);
-        }
-    }
-
-    @Override
-    @AuthorityProviderContext
-    public void setApplicationContext(final ApplicationContext applicationContext) throws BeansException {
-        this.applicationContext = applicationContext;
-    }
-
-    @AuthorityProviderContext
-    public void setNiFiProperties(NiFiProperties properties) {
-        this.properties = properties;
-    }
-
-    private InetSocketAddress getServiceAddress() {
-        final DiscoverableService service = serviceLocator.getService();
-        if (service != null) {
-            return service.getServiceAddress();
-        }
-        return null;
-    }
-
-    private boolean isException(final Class<? extends Exception> exception, final ProtocolMessage protocolMessage) {
-        if (protocolMessage.wasException()) {
-            return exception.getName().equals(protocolMessage.getExceptionClass());
-        } else {
-            return false;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/DoesDnExistMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/DoesDnExistMessage.java b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/DoesDnExistMessage.java
deleted file mode 100644
index 38d0dd8..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/DoesDnExistMessage.java
+++ /dev/null
@@ -1,56 +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.cluster.authorization.protocol.message;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage.MessageType;
-
-/**
- * @author unattributed
- */
-@XmlRootElement(name = "doesDnExistMessage")
-public class DoesDnExistMessage extends ProtocolMessage {
-
-    private String dn;
-
-    private boolean response;
-
-    public DoesDnExistMessage() {
-    }
-
-    @Override
-    public MessageType getType() {
-        return MessageType.DOES_DN_EXIST;
-    }
-
-    public String getDn() {
-        return dn;
-    }
-
-    public void setDn(String dn) {
-        this.dn = dn;
-    }
-
-    public boolean getResponse() {
-        return response;
-    }
-
-    public void setResponse(boolean response) {
-        this.response = response;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetAuthoritiesMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetAuthoritiesMessage.java b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetAuthoritiesMessage.java
deleted file mode 100644
index 347163f..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetAuthoritiesMessage.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.cluster.authorization.protocol.message;
-
-import java.util.HashSet;
-import java.util.Set;
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.authorization.Authority;
-
-/**
- * @author unattributed
- */
-@XmlRootElement(name = "getAuthoritiesMessage")
-public class GetAuthoritiesMessage extends ProtocolMessage {
-
-    private String dn;
-
-    private Set<Authority> response = new HashSet<>();
-
-    public GetAuthoritiesMessage() {
-    }
-
-    @Override
-    public MessageType getType() {
-        return MessageType.GET_AUTHORITIES;
-    }
-
-    public String getDn() {
-        return dn;
-    }
-
-    public void setDn(String dn) {
-        this.dn = dn;
-    }
-
-    public Set<Authority> getResponse() {
-        return response;
-    }
-
-    public void setResponse(Set<Authority> response) {
-        this.response = response;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetGroupForUserMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetGroupForUserMessage.java b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetGroupForUserMessage.java
deleted file mode 100644
index 717f244..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetGroupForUserMessage.java
+++ /dev/null
@@ -1,55 +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.cluster.authorization.protocol.message;
-
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- * @author unattributed
- */
-@XmlRootElement(name = "getGroupForUserMessage")
-public class GetGroupForUserMessage extends ProtocolMessage {
-
-    private String dn;
-
-    private String response;
-
-    public GetGroupForUserMessage() {
-    }
-
-    @Override
-    public MessageType getType() {
-        return MessageType.GET_GROUP_FOR_USER;
-    }
-
-    public String getDn() {
-        return dn;
-    }
-
-    public void setDn(String dn) {
-        this.dn = dn;
-    }
-
-    public String getResponse() {
-        return response;
-    }
-
-    public void setResponse(String response) {
-        this.response = response;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/ProtocolMessage.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/ProtocolMessage.java b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/ProtocolMessage.java
deleted file mode 100644
index 102142a..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/ProtocolMessage.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.cluster.authorization.protocol.message;
-
-/**
- * @author unattributed
- */
-public abstract class ProtocolMessage {
-
-    private String exceptionClass;
-    private String exceptionMessage;
-
-    public static enum MessageType {
-
-        DOES_DN_EXIST,
-        GET_AUTHORITIES,
-        GET_USERS,
-        GET_GROUP_FOR_USER
-    }
-
-    public abstract MessageType getType();
-
-    public boolean wasException() {
-        return exceptionClass != null;
-    }
-
-    public String getExceptionMessage() {
-        return exceptionMessage;
-    }
-
-    public void setExceptionMessage(final String exceptionMessage) {
-        this.exceptionMessage = exceptionMessage;
-    }
-
-    public String getExceptionClass() {
-        return exceptionClass;
-    }
-
-    public void setExceptionClass(String exceptionClass) {
-        this.exceptionClass = exceptionClass;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/JaxbProtocolUtils.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/JaxbProtocolUtils.java b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/JaxbProtocolUtils.java
deleted file mode 100644
index 97a1bc7..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/JaxbProtocolUtils.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.cluster.authorization.protocol.message.jaxb;
-
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-
-/**
- * @author unattributed
- */
-public final class JaxbProtocolUtils {
-
-    public static final String JAXB_CONTEXT_PATH = ObjectFactory.class.getPackage().getName();
-
-    public static final JAXBContext JAXB_CONTEXT = initializeJaxbContext();
-
-    /**
-     * Load the JAXBContext version.
-     */
-    private static JAXBContext initializeJaxbContext() {
-        try {
-            return JAXBContext.newInstance(JAXB_CONTEXT_PATH);
-        } catch (JAXBException e) {
-            throw new RuntimeException("Unable to create JAXBContext.");
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/ObjectFactory.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/ObjectFactory.java b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/ObjectFactory.java
deleted file mode 100644
index 5cde335..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/ObjectFactory.java
+++ /dev/null
@@ -1,45 +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.cluster.authorization.protocol.message.jaxb;
-
-import javax.xml.bind.annotation.XmlRegistry;
-import org.apache.nifi.cluster.authorization.protocol.message.DoesDnExistMessage;
-import org.apache.nifi.cluster.authorization.protocol.message.GetAuthoritiesMessage;
-import org.apache.nifi.cluster.authorization.protocol.message.GetGroupForUserMessage;
-
-/**
- * @author unattributed
- */
-@XmlRegistry
-public class ObjectFactory {
-
-    public ObjectFactory() {
-    }
-
-    public DoesDnExistMessage createDoesDnExistMessage() {
-        return new DoesDnExistMessage();
-    }
-
-    public GetAuthoritiesMessage createGetAuthoritiesMessage() {
-        return new GetAuthoritiesMessage();
-    }
-
-    public GetGroupForUserMessage createGetGroupForUserMessage() {
-        return new GetGroupForUserMessage();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider
deleted file mode 100644
index 56f4c3e..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider
+++ /dev/null
@@ -1,16 +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.
-org.apache.nifi.cluster.authorization.ClusterManagerAuthorizationProvider
-org.apache.nifi.cluster.authorization.NodeAuthorizationProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/.gitignore
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/.gitignore b/nar-bundles/framework-bundle/framework/cluster-protocol/.gitignore
deleted file mode 100755
index ea8c4bf..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/.gitignore
+++ /dev/null
@@ -1 +0,0 @@
-/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/pom.xml b/nar-bundles/framework-bundle/framework/cluster-protocol/pom.xml
deleted file mode 100644
index 5351085..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/pom.xml
+++ /dev/null
@@ -1,69 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-framework-parent</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-    <artifactId>framework-cluster-protocol</artifactId>
-    <packaging>jar</packaging>
-    <name>NiFi Framework Cluster Protocol</name>
-    <description>The messaging protocol for clustered NiFi</description>
-    <dependencies>
-        
-        <!-- application dependencies -->
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-properties</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-logging-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-socket-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-security</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>core-api</artifactId>
-        </dependency>
-        
-        <!-- spring dependencies -->
-        <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-core</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-beans</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-context</artifactId>
-        </dependency>
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java
deleted file mode 100644
index fa1547f..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java
+++ /dev/null
@@ -1,69 +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.cluster.protocol;
-
-import org.apache.nifi.cluster.protocol.message.DisconnectMessage;
-import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
-import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
-import org.apache.nifi.cluster.protocol.message.PrimaryRoleAssignmentMessage;
-import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
-import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage;
-import org.apache.nifi.reporting.BulletinRepository;
-
-/**
- * An interface for sending protocol messages from the cluster manager to nodes.
- * 
- * @author unattributed
- */
-public interface ClusterManagerProtocolSender {
-    
-    /**
-     * Sends a "flow request" message to a node.
-     * @param msg a message
-     * @return the response
-     * @throws ProtocolException if communication failed 
-     */
-    FlowResponseMessage requestFlow(FlowRequestMessage msg) throws ProtocolException;
-
-    /**
-     * Sends a "reconnection request" message to a node.
-     * @param msg a message
-     * @return 
-     * @throws ProtocolException if communication failed
-     */
-    ReconnectionResponseMessage requestReconnection(ReconnectionRequestMessage msg) throws ProtocolException;
-    
-    /**
-     * Sends a "disconnection request" message to a node.
-     * @param msg a message
-     * @throws ProtocolException if communication failed
-     */
-    void disconnect(DisconnectMessage msg) throws ProtocolException;
-    
-    /**
-     * Sends an "assign primary role" message to a node.
-     * @param msg a message
-     * @throws ProtocolException if communication failed 
-     */
-    void assignPrimaryRole(PrimaryRoleAssignmentMessage msg) throws ProtocolException;
-
-    /**
-     * Sets the {@link BulletinRepository} that can be used to report bulletins
-     * @param bulletinRepository
-     */
-    void setBulletinRepository(final BulletinRepository bulletinRepository);
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionRequest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionRequest.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionRequest.java
deleted file mode 100644
index 1b5d007..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionRequest.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.cluster.protocol;
-
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.cluster.protocol.jaxb.message.ConnectionRequestAdapter;
-
-/**
- * A node's request to connect to the cluster.  The request contains a proposed
- * identifier.
- * 
- * @author unattributed
- */
-@XmlJavaTypeAdapter(ConnectionRequestAdapter.class)
-public class ConnectionRequest {
-
-    private final NodeIdentifier proposedNodeIdentifier;
-
-    public ConnectionRequest(final NodeIdentifier proposedNodeIdentifier) {
-        if(proposedNodeIdentifier == null) {
-            throw new IllegalArgumentException("Proposed node identifier may not be null.");
-        }
-        this.proposedNodeIdentifier = proposedNodeIdentifier;
-    }
-
-    public NodeIdentifier getProposedNodeIdentifier() {
-        return proposedNodeIdentifier;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java
deleted file mode 100644
index 7a5ff2b..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java
+++ /dev/null
@@ -1,141 +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.cluster.protocol;
-
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-
-import org.apache.nifi.cluster.protocol.jaxb.message.ConnectionResponseAdapter;
-
-/**
- * The cluster manager's response to a node's connection request.  If the manager
- * has a current copy of the data flow, then it is returned with a node identifier
- * to the node.  Otherwise, the manager will provide a "try again in X seconds" 
- * response to the node in hopes that a current data flow will be available upon
- * subsequent requests.
- * 
- * @author unattributed
- */
-@XmlJavaTypeAdapter(ConnectionResponseAdapter.class)
-public class ConnectionResponse {
-
-    private final boolean blockedByFirewall;
-    private final int tryLaterSeconds;
-    private final NodeIdentifier nodeIdentifier;
-    private final StandardDataFlow dataFlow;
-    private final boolean primary;
-    private final Integer managerRemoteInputPort;
-    private final Boolean managerRemoteCommsSecure;
-    private final String instanceId;
-    
-    private volatile String clusterManagerDN;
-    
-    public ConnectionResponse(final NodeIdentifier nodeIdentifier, final StandardDataFlow dataFlow, final boolean primary, 
-        final Integer managerRemoteInputPort, final Boolean managerRemoteCommsSecure, final String instanceId) {
-        if(nodeIdentifier == null) {
-            throw new IllegalArgumentException("Node identifier may not be empty or null.");
-        } else if(dataFlow == null) {
-            throw new IllegalArgumentException("DataFlow may not be null.");
-        }
-        this.nodeIdentifier = nodeIdentifier;
-        this.dataFlow = dataFlow;
-        this.tryLaterSeconds = 0;
-        this.blockedByFirewall = false;
-        this.primary = primary;
-        this.managerRemoteInputPort = managerRemoteInputPort;
-        this.managerRemoteCommsSecure = managerRemoteCommsSecure;
-        this.instanceId = instanceId;
-    }
-    
-    public ConnectionResponse(final int tryLaterSeconds) {
-        if(tryLaterSeconds <= 0) {
-            throw new IllegalArgumentException("Try-Later seconds may not be nonnegative: " + tryLaterSeconds);
-        }
-        this.dataFlow = null;
-        this.nodeIdentifier = null;
-        this.tryLaterSeconds = tryLaterSeconds;
-        this.blockedByFirewall = false;
-        this.primary = false;
-        this.managerRemoteInputPort = null;
-        this.managerRemoteCommsSecure = null;
-        this.instanceId = null;
-    }
-
-    private ConnectionResponse() {
-        this.dataFlow = null;
-        this.nodeIdentifier = null;
-        this.tryLaterSeconds = 0;
-        this.blockedByFirewall = true;
-        this.primary = false;
-        this.managerRemoteInputPort = null;
-        this.managerRemoteCommsSecure = null;
-        this.instanceId = null;
-    }
-    
-    public static ConnectionResponse createBlockedByFirewallResponse() {
-        return new ConnectionResponse();
-    }
-    
-    public boolean isPrimary() {
-        return primary;
-    }
-    
-    public boolean shouldTryLater() {
-        return tryLaterSeconds > 0;
-    }
-    
-    public boolean isBlockedByFirewall() {
-        return blockedByFirewall;
-    }
-
-    public int getTryLaterSeconds() {
-        return tryLaterSeconds;
-    }
-    
-    public StandardDataFlow getDataFlow() {
-        return dataFlow;
-    }
-    
-    public NodeIdentifier getNodeIdentifier() {
-        return nodeIdentifier;
-    }
-
-    public Integer getManagerRemoteInputPort() {
-        return managerRemoteInputPort;
-    }
-    
-    public Boolean isManagerRemoteCommsSecure() {
-        return managerRemoteCommsSecure;
-    }
-    
-    public String getInstanceId() {
-        return instanceId;
-    }
-    
-    public void setClusterManagerDN(final String dn) {
-        this.clusterManagerDN = dn;
-    }
-    
-    /**
-     * Returns the DN of the NCM, if it is available or <code>null</code> otherwise.
-     * 
-     * @return
-     */
-    public String getClusterManagerDN() {
-        return clusterManagerDN;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java
deleted file mode 100644
index 67324a1..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java
+++ /dev/null
@@ -1,68 +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.cluster.protocol;
-
-import java.util.Date;
-import javax.xml.bind.annotation.XmlTransient;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.cluster.protocol.jaxb.message.HeartbeatAdapter;
-
-/**
- * A heartbeat for indicating the status of a node to the cluster.
- * @author unattributed
- */
-@XmlJavaTypeAdapter(HeartbeatAdapter.class)
-public class Heartbeat {
-    
-    private final NodeIdentifier nodeIdentifier;
-    private final boolean primary;
-    private final boolean connected;
-    private final long createdTimestamp;
-    private final byte[] payload;
-    
-    public Heartbeat(final NodeIdentifier nodeIdentifier, final boolean primary, final boolean connected, final byte[] payload) {
-        if(nodeIdentifier == null) {
-            throw new IllegalArgumentException("Node Identifier may not be null.");
-        } 
-        this.nodeIdentifier = nodeIdentifier;
-        this.primary = primary;
-        this.connected = connected;
-        this.payload = payload;
-        this.createdTimestamp = new Date().getTime();
-    }
-    
-    public NodeIdentifier getNodeIdentifier() {
-        return nodeIdentifier;
-    }
-    
-    public byte[] getPayload() {
-        return payload;
-    }
-    
-    public boolean isPrimary() {
-        return primary;
-    }
-    
-    public boolean isConnected() {
-        return connected;
-    }
-    
-    @XmlTransient
-    public long getCreatedTimestamp() {
-        return createdTimestamp;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeBulletins.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeBulletins.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeBulletins.java
deleted file mode 100644
index a120524..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeBulletins.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.cluster.protocol;
-
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.cluster.protocol.jaxb.message.NodeBulletinsAdapter;
-
-/**
- *
- */
-@XmlJavaTypeAdapter(NodeBulletinsAdapter.class)
-public class NodeBulletins {
-
-    private final NodeIdentifier nodeIdentifier;
-    private final byte[] payload;
-
-    public NodeBulletins(NodeIdentifier nodeIdentifier, byte[] payload) {
-        this.nodeIdentifier = nodeIdentifier;
-        this.payload = payload;
-    }
-
-    public NodeIdentifier getNodeIdentifier() {
-        return nodeIdentifier;
-    }
-
-    public byte[] getPayload() {
-        return payload;
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java
deleted file mode 100644
index 1893186..0000000
--- a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java
+++ /dev/null
@@ -1,172 +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.cluster.protocol;
-
-import org.apache.commons.lang3.StringUtils;
-
-/**
- * A node identifier denoting the coordinates of a flow controller that is connected 
- * to a cluster.  Nodes provide an external public API interface and an internal private
- * interface for communicating with the cluster.
- * 
- * The external API interface and internal protocol each require an IP or hostname 
- * as well as a port for communicating. 
- * 
- * This class overrides hashCode and equals and considers two instances to be
- * equal if they have the equal IDs.
- * 
- * @author unattributed
- * @Immutable
- * @Threadsafe
- */
-public class NodeIdentifier {
- 
-    /** the unique identifier for the node */
-    private final String id;
-    
-    /** the IP or hostname to use for sending requests to the node's external interface */
-    private final String apiAddress;
-    
-    /** the port to use use for sending requests to the node's external interface */
-    private final int apiPort;    
-    
-    /** the IP or hostname to use for sending requests to the node's internal interface */
-    private final String socketAddress;
-    
-    /** the port to use use for sending requests to the node's internal interface */
-    private final int socketPort;
-    
-    private final String nodeDn;
-
-    public NodeIdentifier(final String id, final String apiAddress, final int apiPort, final String socketAddress, final int socketPort) {
-        this(id, apiAddress, apiPort, socketAddress, socketPort, null);
-    }
-    
-    public NodeIdentifier(final String id, final String apiAddress, final int apiPort, final String socketAddress, final int socketPort, final String dn) {
-        
-        if(StringUtils.isBlank(id)) {
-            throw new IllegalArgumentException("Node ID may not be empty or null.");
-        } else if(StringUtils.isBlank(apiAddress)) {
-            throw new IllegalArgumentException("Node API address may not be empty or null.");
-        } else if(StringUtils.isBlank(socketAddress)) {
-            throw new IllegalArgumentException("Node socket address may not be empty or null.");
-        } 
-        
-        validatePort(apiPort);
-        validatePort(socketPort);
-        
-        this.id = id;
-        this.apiAddress = apiAddress;
-        this.apiPort = apiPort;
-        this.socketAddress = socketAddress;
-        this.socketPort = socketPort;
-        this.nodeDn = dn;
-    }
-
-    public String getId() {
-        return id;
-    }
-    
-    public String getDN() {
-        return nodeDn;
-    }
-    
-    public String getApiAddress() {
-        return apiAddress;
-    }
-
-    public int getApiPort() {
-        return apiPort;
-    }
-
-    public String getSocketAddress() {
-        return socketAddress;
-    }
-    
-    public int getSocketPort() {
-        return socketPort;
-    }
-    
-    private void validatePort(final int port) {
-        if(port < 1 || port > 65535) {
-            throw new IllegalArgumentException("Port must be inclusively in the range [1, 65535].  Port given: " + port);
-        }   
-    }
-    
-    /**
-     * Compares the id of two node identifiers for equality.
-     * 
-     * @param obj a node identifier
-     * 
-     * @return true if the id is equal; false otherwise
-     */
-    @Override
-    public boolean equals(Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (getClass() != obj.getClass()) {
-            return false;
-        }
-        final NodeIdentifier other = (NodeIdentifier) obj;
-        if ((this.id == null) ? (other.id != null) : !this.id.equals(other.id)) {
-            return false;
-        }
-        return true;
-    }
-
-    /**
-     * Compares API address/port and socket address/port for equality.  The 
-     * id is not used for comparison.
-     * 
-     * @param other a node identifier
-     * 
-     * @return true if API address/port and socket address/port are equal; false
-     * otherwise
-     */
-    public boolean logicallyEquals(final NodeIdentifier other) {
-        if(other == null) {
-            return false;
-        }
-        if ((this.apiAddress == null) ? (other.apiAddress != null) : !this.apiAddress.equals(other.apiAddress)) {
-            return false;
-        }
-        if(this.apiPort != other.apiPort) {
-            return false;
-        }
-        if ((this.socketAddress == null) ? (other.socketAddress != null) : !this.socketAddress.equals(other.socketAddress)) {
-            return false;
-        }
-        if(this.socketPort != other.socketPort) {
-            return false;
-        }
-        return true;
-    }
-    
-    @Override
-    public int hashCode() {
-        int hash = 7;
-        hash = 31 * hash + (this.id != null ? this.id.hashCode() : 0);
-        return hash;
-    }
-
-    @Override
-    public String toString() {
-        return "[" + "id=" + id + ", apiAddress=" + apiAddress + ", apiPort=" + apiPort + ", socketAddress=" + socketAddress + ", socketPort=" + socketPort + ']';
-    }
-
-}


[05/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowController.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowController.java
deleted file mode 100644
index 346e801..0000000
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ /dev/null
@@ -1,3579 +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;
-
-import static java.util.Objects.requireNonNull;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import javax.net.ssl.SSLContext;
-
-import org.apache.nifi.admin.service.UserService;
-import org.apache.nifi.cluster.BulletinsPayload;
-import org.apache.nifi.cluster.HeartbeatPayload;
-import org.apache.nifi.cluster.protocol.DataFlow;
-import org.apache.nifi.cluster.protocol.Heartbeat;
-import org.apache.nifi.cluster.protocol.NodeBulletins;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.NodeProtocolSender;
-import org.apache.nifi.cluster.protocol.UnknownServiceAddressException;
-import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
-import org.apache.nifi.cluster.protocol.message.NodeBulletinsMessage;
-import org.apache.nifi.connectable.Connectable;
-import org.apache.nifi.connectable.ConnectableType;
-import org.apache.nifi.connectable.Connection;
-import org.apache.nifi.connectable.Funnel;
-import org.apache.nifi.connectable.LocalPort;
-import org.apache.nifi.connectable.Port;
-import org.apache.nifi.connectable.Position;
-import org.apache.nifi.connectable.Size;
-import org.apache.nifi.connectable.StandardConnection;
-import org.apache.nifi.controller.exception.CommunicationsException;
-import org.apache.nifi.controller.exception.ProcessorInstantiationException;
-import org.apache.nifi.controller.exception.ProcessorLifeCycleException;
-import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.label.StandardLabel;
-import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
-import org.apache.nifi.controller.reporting.StandardReportingTaskNode;
-import org.apache.nifi.controller.repository.ContentRepository;
-import org.apache.nifi.controller.repository.CounterRepository;
-import org.apache.nifi.controller.repository.FlowFileEvent;
-import org.apache.nifi.controller.repository.FlowFileEventRepository;
-import org.apache.nifi.controller.repository.FlowFileRecord;
-import org.apache.nifi.controller.repository.FlowFileRepository;
-import org.apache.nifi.controller.repository.FlowFileSwapManager;
-import org.apache.nifi.controller.repository.QueueProvider;
-import org.apache.nifi.controller.repository.RepositoryRecord;
-import org.apache.nifi.controller.repository.RepositoryStatusReport;
-import org.apache.nifi.controller.repository.StandardCounterRepository;
-import org.apache.nifi.controller.repository.StandardFlowFileRecord;
-import org.apache.nifi.controller.repository.StandardRepositoryRecord;
-import org.apache.nifi.controller.repository.claim.ContentClaim;
-import org.apache.nifi.controller.repository.claim.ContentClaimManager;
-import org.apache.nifi.controller.repository.claim.ContentDirection;
-import org.apache.nifi.controller.repository.claim.StandardContentClaimManager;
-import org.apache.nifi.controller.repository.io.LimitedInputStream;
-import org.apache.nifi.controller.scheduling.EventDrivenSchedulingAgent;
-import org.apache.nifi.controller.scheduling.ProcessContextFactory;
-import org.apache.nifi.controller.scheduling.QuartzSchedulingAgent;
-import org.apache.nifi.controller.scheduling.StandardProcessScheduler;
-import org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent;
-import org.apache.nifi.controller.service.ControllerServiceNode;
-import org.apache.nifi.controller.service.ControllerServiceProvider;
-import org.apache.nifi.controller.service.StandardControllerServiceProvider;
-import org.apache.nifi.controller.status.ConnectionStatus;
-import org.apache.nifi.controller.status.PortStatus;
-import org.apache.nifi.controller.status.ProcessGroupStatus;
-import org.apache.nifi.controller.status.ProcessorStatus;
-import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
-import org.apache.nifi.controller.status.RunStatus;
-import org.apache.nifi.controller.status.TransmissionStatus;
-import org.apache.nifi.controller.status.history.ComponentStatusRepository;
-import org.apache.nifi.controller.status.history.StatusHistoryUtil;
-import org.apache.nifi.controller.tasks.ExpireFlowFiles;
-import org.apache.nifi.diagnostics.SystemDiagnostics;
-import org.apache.nifi.diagnostics.SystemDiagnosticsFactory;
-import org.apache.nifi.encrypt.StringEncryptor;
-import org.apache.nifi.engine.FlowEngine;
-import org.apache.nifi.events.BulletinFactory;
-import org.apache.nifi.events.EventReporter;
-import org.apache.nifi.events.NodeBulletinProcessingStrategy;
-import org.apache.nifi.events.VolatileBulletinRepository;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.framework.security.util.SslContextFactory;
-import org.apache.nifi.groups.ProcessGroup;
-import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.groups.RemoteProcessGroupPortDescriptor;
-import org.apache.nifi.groups.StandardProcessGroup;
-import org.apache.nifi.stream.io.StreamUtils;
-import org.apache.nifi.logging.LogLevel;
-import org.apache.nifi.logging.LogRepository;
-import org.apache.nifi.logging.LogRepositoryFactory;
-import org.apache.nifi.logging.ProcessorLog;
-import org.apache.nifi.logging.ProcessorLogObserver;
-import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.nar.NarCloseable;
-import org.apache.nifi.nar.NarThreadContextClassLoader;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.ProcessorInitializationContext;
-import org.apache.nifi.processor.QueueSize;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.processor.SimpleProcessLogger;
-import org.apache.nifi.processor.StandardProcessorInitializationContext;
-import org.apache.nifi.processor.StandardValidationContextFactory;
-import org.apache.nifi.processor.annotation.OnAdded;
-import org.apache.nifi.provenance.ProvenanceEventRecord;
-import org.apache.nifi.provenance.ProvenanceEventRepository;
-import org.apache.nifi.provenance.ProvenanceEventType;
-import org.apache.nifi.provenance.StandardProvenanceEventRecord;
-import org.apache.nifi.remote.RemoteGroupPort;
-import org.apache.nifi.remote.RemoteResourceManager;
-import org.apache.nifi.remote.RemoteSiteListener;
-import org.apache.nifi.remote.RootGroupPort;
-import org.apache.nifi.remote.SocketRemoteSiteListener;
-import org.apache.nifi.remote.StandardRemoteProcessGroup;
-import org.apache.nifi.remote.StandardRemoteProcessGroupPortDescriptor;
-import org.apache.nifi.remote.StandardRootGroupPort;
-import org.apache.nifi.remote.TransferDirection;
-import org.apache.nifi.remote.protocol.socket.SocketFlowFileServerProtocol;
-import org.apache.nifi.reporting.Bulletin;
-import org.apache.nifi.reporting.BulletinRepository;
-import org.apache.nifi.reporting.EventAccess;
-import org.apache.nifi.reporting.ReportingTask;
-import org.apache.nifi.reporting.Severity;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.util.ReflectionUtils;
-import org.apache.nifi.web.api.dto.ConnectableDTO;
-import org.apache.nifi.web.api.dto.ConnectionDTO;
-import org.apache.nifi.web.api.dto.FlowSnippetDTO;
-import org.apache.nifi.web.api.dto.FunnelDTO;
-import org.apache.nifi.web.api.dto.LabelDTO;
-import org.apache.nifi.web.api.dto.PortDTO;
-import org.apache.nifi.web.api.dto.PositionDTO;
-import org.apache.nifi.web.api.dto.ProcessGroupDTO;
-import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
-import org.apache.nifi.web.api.dto.ProcessorDTO;
-import org.apache.nifi.web.api.dto.RelationshipDTO;
-import org.apache.nifi.web.api.dto.RemoteProcessGroupContentsDTO;
-import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
-import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
-import org.apache.nifi.web.api.dto.TemplateDTO;
-import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
-import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.sun.jersey.api.client.ClientHandlerException;
-
-public class FlowController implements EventAccess, ControllerServiceProvider, Heartbeater, QueueProvider {
-
-    // default repository implementations
-    public static final String DEFAULT_FLOWFILE_REPO_IMPLEMENTATION = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository";
-    public static final String DEFAULT_CONTENT_REPO_IMPLEMENTATION = "org.apache.nifi.controller.repository.FileSystemRepository";
-    public static final String DEFAULT_PROVENANCE_REPO_IMPLEMENTATION = "org.apache.nifi.provenance.VolatileProvenanceRepository";
-    public static final String DEFAULT_SWAP_MANAGER_IMPLEMENTATION = "org.apache.nifi.controller.FileSystemSwapManager";
-    public static final String DEFAULT_COMPONENT_STATUS_REPO_IMPLEMENTATION = "org.apache.nifi.controller.status.history.VolatileComponentStatusRepository";
-
-    public static final String SCHEDULE_MINIMUM_NANOSECONDS = "flowcontroller.minimum.nanoseconds";
-    public static final String GRACEFUL_SHUTDOWN_PERIOD = "nifi.flowcontroller.graceful.shutdown.seconds";
-    public static final long DEFAULT_GRACEFUL_SHUTDOWN_SECONDS = 10;
-    public static final int METRICS_RESERVOIR_SIZE = 288;   // 1 day worth of 5-minute captures
-
-    public static final String ROOT_GROUP_ID_ALIAS = "root";
-    public static final String DEFAULT_ROOT_GROUP_NAME = "NiFi Flow";
-
-    private final AtomicInteger maxTimerDrivenThreads;
-    private final AtomicInteger maxEventDrivenThreads;
-    private final AtomicReference<FlowEngine> timerDrivenEngineRef;
-    private final AtomicReference<FlowEngine> eventDrivenEngineRef;
-
-    private final ContentRepository contentRepository;
-    private final FlowFileRepository flowFileRepository;
-    private final FlowFileEventRepository flowFileEventRepository;
-    private final ProvenanceEventRepository provenanceEventRepository;
-    private final VolatileBulletinRepository bulletinRepository;
-    private final StandardProcessScheduler processScheduler;
-    private final TemplateManager templateManager;
-    private final SnippetManager snippetManager;
-    private final long gracefulShutdownSeconds;
-    private final ExtensionManager extensionManager;
-    private final NiFiProperties properties;
-    private final SSLContext sslContext;
-    private final RemoteSiteListener externalSiteListener;
-    private final AtomicReference<CounterRepository> counterRepositoryRef;
-    private final AtomicBoolean initialized = new AtomicBoolean(false);
-    private final ControllerServiceProvider controllerServiceProvider;
-    private final UserService userService;
-    private final EventDrivenWorkerQueue eventDrivenWorkerQueue;
-    private final ComponentStatusRepository componentStatusRepository;
-    private final long systemStartTime = System.currentTimeMillis();    // time at which the node was started
-    private final ConcurrentMap<String, ReportingTaskNode> reportingTasks = new ConcurrentHashMap<>();
-
-    // The Heartbeat Bean is used to provide an Atomic Reference to data that is used in heartbeats that may
-    // change while the instance is running. We do this because we want to generate heartbeats even if we
-    // are unable to obtain a read lock on the entire FlowController.
-    private final AtomicReference<HeartbeatBean> heartbeatBeanRef = new AtomicReference<>();
-    private final AtomicBoolean heartbeatsSuspended = new AtomicBoolean(false);
-
-    private final Integer remoteInputSocketPort;
-    private final Boolean isSiteToSiteSecure;
-    private Integer clusterManagerRemoteSitePort = null;
-    private Boolean clusterManagerRemoteSiteCommsSecure = null;
-
-    private ProcessGroup rootGroup;
-    private final List<Connectable> startConnectablesAfterInitialization;
-    private final List<RemoteGroupPort> startRemoteGroupPortsAfterInitialization;
-
-    /**
-     * true if controller is configured to operate in a clustered environment
-     */
-    private final boolean configuredForClustering;
-
-    /**
-     * the time to wait between heartbeats
-     */
-    private final int heartbeatDelaySeconds;
-
-    /**
-     * The sensitive property string encryptor *
-     */
-    private final StringEncryptor encryptor;
-
-    /**
-     * cluster protocol sender
-     */
-    private final NodeProtocolSender protocolSender;
-
-    private final ScheduledExecutorService clusterTaskExecutor = new FlowEngine(3, "Clustering Tasks");
-    private final ContentClaimManager contentClaimManager = new StandardContentClaimManager();
-
-    // guarded by rwLock
-    /**
-     * timer to periodically send heartbeats to the cluster
-     */
-    private ScheduledFuture<?> bulletinFuture;
-    private ScheduledFuture<?> heartbeatGeneratorFuture;
-    private ScheduledFuture<?> heartbeatSenderFuture;
-
-    // guarded by FlowController lock
-    /**
-     * timer task to generate heartbeats
-     */
-    private final AtomicReference<HeartbeatMessageGeneratorTask> heartbeatMessageGeneratorTaskRef = new AtomicReference<>(null);
-
-    private AtomicReference<NodeBulletinProcessingStrategy> nodeBulletinSubscriber;
-
-    // guarded by rwLock
-    /**
-     * the node identifier;
-     */
-    private NodeIdentifier nodeId;
-
-    // guarded by rwLock
-    /**
-     * true if controller is connected or trying to connect to the cluster
-     */
-    private boolean clustered;
-    private String clusterManagerDN;
-
-    // guarded by rwLock
-    /**
-     * true if controller is the primary of the cluster
-     */
-    private boolean primary;
-
-    // guarded by rwLock
-    /**
-     * true if connected to a cluster
-     */
-    private boolean connected;
-
-    // guarded by rwLock
-    private String instanceId;
-
-    private volatile boolean shutdown = false;
-
-    private final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
-    private final Lock readLock = rwLock.readLock();
-    private final Lock writeLock = rwLock.writeLock();
-
-    private FlowFileSwapManager flowFileSwapManager;    // guarded by read/write lock
-
-    private static final Logger LOG = LoggerFactory.getLogger(FlowController.class);
-    private static final Logger heartbeatLogger = LoggerFactory.getLogger("org.apache.nifi.cluster.heartbeat");
-
-    public static FlowController createStandaloneInstance(
-            final FlowFileEventRepository flowFileEventRepo,
-            final NiFiProperties properties,
-            final UserService userService,
-            final StringEncryptor encryptor) {
-        return new FlowController(
-                flowFileEventRepo,
-                properties,
-                userService,
-                encryptor,
-                /* configuredForClustering */ false,
-                /* NodeProtocolSender */ null);
-    }
-
-    public static FlowController createClusteredInstance(
-            final FlowFileEventRepository flowFileEventRepo,
-            final NiFiProperties properties,
-            final UserService userService,
-            final StringEncryptor encryptor,
-            final NodeProtocolSender protocolSender) {
-        final FlowController flowController = new FlowController(
-                flowFileEventRepo,
-                properties,
-                userService,
-                encryptor,
-                /* configuredForClustering */ true,
-                /* NodeProtocolSender */ protocolSender);
-
-        flowController.setClusterManagerRemoteSiteInfo(properties.getRemoteInputPort(), properties.isSiteToSiteSecure());
-
-        return flowController;
-    }
-
-    private FlowController(
-            final FlowFileEventRepository flowFileEventRepo,
-            final NiFiProperties properties,
-            final UserService userService,
-            final StringEncryptor encryptor,
-            final boolean configuredForClustering,
-            final NodeProtocolSender protocolSender) {
-
-        maxTimerDrivenThreads = new AtomicInteger(10);
-        maxEventDrivenThreads = new AtomicInteger(5);
-
-        this.encryptor = encryptor;
-        this.properties = properties;
-        sslContext = SslContextFactory.createSslContext(properties, false);
-        extensionManager = new ExtensionManager();
-        controllerServiceProvider = new StandardControllerServiceProvider();
-
-        timerDrivenEngineRef = new AtomicReference<>(new FlowEngine(maxTimerDrivenThreads.get(), "Timer-Driven Process"));
-        eventDrivenEngineRef = new AtomicReference<>(new FlowEngine(maxEventDrivenThreads.get(), "Event-Driven Process"));
-
-        final FlowFileRepository flowFileRepo = createFlowFileRepository(properties, contentClaimManager);
-        flowFileRepository = flowFileRepo;
-        flowFileEventRepository = flowFileEventRepo;
-        counterRepositoryRef = new AtomicReference<CounterRepository>(new StandardCounterRepository());
-
-        bulletinRepository = new VolatileBulletinRepository();
-        nodeBulletinSubscriber = new AtomicReference<>();
-
-        try {
-            this.provenanceEventRepository = createProvenanceRepository(properties);
-            this.provenanceEventRepository.initialize(createEventReporter(bulletinRepository));
-
-            this.contentRepository = createContentRepository(properties);
-        } catch (final Exception e) {
-            throw new RuntimeException("Unable to create Provenance Repository", e);
-        }
-
-        processScheduler = new StandardProcessScheduler(this, this, encryptor);
-        eventDrivenWorkerQueue = new EventDrivenWorkerQueue(false, false, processScheduler);
-
-        final ProcessContextFactory contextFactory = new ProcessContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceEventRepository);
-        processScheduler.setSchedulingAgent(SchedulingStrategy.EVENT_DRIVEN, new EventDrivenSchedulingAgent(
-                eventDrivenEngineRef.get(), this, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor));
-
-        final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor);
-        final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor);
-        processScheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, timerDrivenAgent);
-        processScheduler.setSchedulingAgent(SchedulingStrategy.PRIMARY_NODE_ONLY, timerDrivenAgent);
-        processScheduler.setSchedulingAgent(SchedulingStrategy.CRON_DRIVEN, quartzSchedulingAgent);
-        processScheduler.scheduleFrameworkTask(new ExpireFlowFiles(this, contextFactory), "Expire FlowFiles", 30L, 30L, TimeUnit.SECONDS);
-
-        startConnectablesAfterInitialization = new ArrayList<>();
-        startRemoteGroupPortsAfterInitialization = new ArrayList<>();
-        this.userService = userService;
-
-        final String gracefulShutdownSecondsVal = properties.getProperty(GRACEFUL_SHUTDOWN_PERIOD);
-        long shutdownSecs;
-        try {
-            shutdownSecs = Long.parseLong(gracefulShutdownSecondsVal);
-            if (shutdownSecs < 1) {
-                shutdownSecs = DEFAULT_GRACEFUL_SHUTDOWN_SECONDS;
-            }
-        } catch (final NumberFormatException nfe) {
-            shutdownSecs = DEFAULT_GRACEFUL_SHUTDOWN_SECONDS;
-        }
-        gracefulShutdownSeconds = shutdownSecs;
-
-        remoteInputSocketPort = properties.getRemoteInputPort();
-        isSiteToSiteSecure = properties.isSiteToSiteSecure();
-
-        if (isSiteToSiteSecure && sslContext == null && remoteInputSocketPort != null) {
-            throw new IllegalStateException("NiFi Configured to allow Secure Site-to-Site communications but the Keystore/Truststore properties are not configured");
-        }
-
-        this.configuredForClustering = configuredForClustering;
-        this.heartbeatDelaySeconds = (int) FormatUtils.getTimeDuration(properties.getNodeHeartbeatInterval(), TimeUnit.SECONDS);
-        this.protocolSender = protocolSender;
-        try {
-            this.templateManager = new TemplateManager(properties.getTemplateDirectory());
-        } catch (IOException e) {
-            throw new RuntimeException(e);
-        }
-
-        this.snippetManager = new SnippetManager();
-
-        rootGroup = new StandardProcessGroup(UUID.randomUUID().toString(), this, processScheduler, properties, encryptor);
-        rootGroup.setName(DEFAULT_ROOT_GROUP_NAME);
-        instanceId = UUID.randomUUID().toString();
-
-        if (remoteInputSocketPort == null){
-            LOG.info("Not enabling Site-to-Site functionality because nifi.remote.input.socket.port is not set");
-            externalSiteListener = null;
-        } else if (isSiteToSiteSecure && sslContext == null) {
-            LOG.error("Unable to create Secure Site-to-Site Listener because not all required Keystore/Truststore Properties are set. Site-to-Site functionality will be disabled until this problem is has been fixed.");
-            externalSiteListener = null;
-        } else {
-            // Register the SocketFlowFileServerProtocol as the appropriate resource for site-to-site Server Protocol
-            RemoteResourceManager.setServerProtocolImplementation(SocketFlowFileServerProtocol.RESOURCE_NAME, SocketFlowFileServerProtocol.class);
-            externalSiteListener = new SocketRemoteSiteListener(remoteInputSocketPort, isSiteToSiteSecure ? sslContext : null);
-            externalSiteListener.setRootGroup(rootGroup);
-        }
-
-        // Determine frequency for obtaining component status snapshots
-        final String snapshotFrequency = properties.getProperty(NiFiProperties.COMPONENT_STATUS_SNAPSHOT_FREQUENCY, NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY);
-        long snapshotMillis;
-        try {
-            snapshotMillis = FormatUtils.getTimeDuration(snapshotFrequency, TimeUnit.MILLISECONDS);
-        } catch (final Exception e) {
-            snapshotMillis = FormatUtils.getTimeDuration(NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY, TimeUnit.MILLISECONDS);
-        }
-
-        componentStatusRepository = createComponentStatusRepository();
-        timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() {
-            @Override
-            public void run() {
-                componentStatusRepository.capture(getControllerStatus());
-            }
-        }, snapshotMillis, snapshotMillis, TimeUnit.MILLISECONDS);
-
-        heartbeatBeanRef.set(new HeartbeatBean(rootGroup, false, false));
-    }
-
-    private static FlowFileRepository createFlowFileRepository(final NiFiProperties properties, final ContentClaimManager contentClaimManager) {
-        final String implementationClassName = properties.getProperty(NiFiProperties.FLOWFILE_REPOSITORY_IMPLEMENTATION, DEFAULT_FLOWFILE_REPO_IMPLEMENTATION);
-        if (implementationClassName == null) {
-            throw new RuntimeException("Cannot create FlowFile Repository because the NiFi Properties is missing the following property: "
-                    + NiFiProperties.FLOWFILE_REPOSITORY_IMPLEMENTATION);
-        }
-
-        try {
-            final FlowFileRepository created = NarThreadContextClassLoader.createInstance(implementationClassName, FlowFileRepository.class);
-            synchronized (created) {
-                created.initialize(contentClaimManager);
-            }
-            return created;
-        } catch (final Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private static FlowFileSwapManager createSwapManager(final NiFiProperties properties) {
-        final String implementationClassName = properties.getProperty(NiFiProperties.FLOWFILE_SWAP_MANAGER_IMPLEMENTATION, DEFAULT_SWAP_MANAGER_IMPLEMENTATION);
-        if (implementationClassName == null) {
-            return null;
-        }
-
-        try {
-            return NarThreadContextClassLoader.createInstance(implementationClassName, FlowFileSwapManager.class);
-        } catch (final Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private static EventReporter createEventReporter(final BulletinRepository bulletinRepository) {
-        return new EventReporter() {
-            @Override
-            public void reportEvent(final Severity severity, final String category, final String message) {
-                final Bulletin bulletin = BulletinFactory.createBulletin(category, severity.name(), message);
-                bulletinRepository.addBulletin(bulletin);
-            }
-        };
-    }
-    
-    public void initializeFlow() throws IOException {
-        writeLock.lock();
-        try {
-            flowFileSwapManager = createSwapManager(properties);
-
-            long maxIdFromSwapFiles = -1L;
-            if (flowFileSwapManager != null) {
-                if (flowFileRepository.isVolatile()) {
-                    flowFileSwapManager.purge();
-                } else {
-                    maxIdFromSwapFiles = flowFileSwapManager.recoverSwappedFlowFiles(this, contentClaimManager);
-                }
-            }
-
-            flowFileRepository.loadFlowFiles(this, maxIdFromSwapFiles + 1);
-
-            // now that we've loaded the FlowFiles, this has restored our ContentClaims' states, so we can tell the
-            // ContentRepository to purge superfluous files
-            contentRepository.cleanup();
-
-            if (flowFileSwapManager != null) {
-                flowFileSwapManager.start(flowFileRepository, this, contentClaimManager, createEventReporter(bulletinRepository));
-            }
-
-            if (externalSiteListener != null) {
-                externalSiteListener.start();
-            }
-
-            timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() {
-                @Override
-                public void run() {
-                    try {
-                        updateRemoteProcessGroups();
-                    } catch (final Throwable t) {
-                        LOG.warn("Unable to update Remote Process Groups due to " + t);
-                        if (LOG.isDebugEnabled()) {
-                            LOG.warn("", t);
-                        }
-                    }
-                }
-            }, 0L, 30L, TimeUnit.SECONDS);
-
-            initialized.set(true);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * <p>
-     * Causes any processors that were added to the flow with a 'delayStart'
-     * flag of true to now start
-     * </p>
-     */
-    public void startDelayed() {
-        writeLock.lock();
-        try {
-            LOG.info("Starting {} processors/ports/funnels", (startConnectablesAfterInitialization.size() + startRemoteGroupPortsAfterInitialization.size()));
-            for (final Connectable connectable : startConnectablesAfterInitialization) {
-                if (connectable.getScheduledState() == ScheduledState.DISABLED) {
-                    continue;
-                }
-
-                try {
-                    if (connectable instanceof ProcessorNode) {
-                        connectable.getProcessGroup().startProcessor((ProcessorNode) connectable);
-                    } else {
-                        startConnectable(connectable);
-                    }
-                } catch (final Throwable t) {
-                    LOG.error("Unable to start {} due to {}", new Object[]{connectable, t});
-                }
-            }
-
-            startConnectablesAfterInitialization.clear();
-
-            int startedTransmitting = 0;
-            for (final RemoteGroupPort remoteGroupPort : startRemoteGroupPortsAfterInitialization) {
-                try {
-                    remoteGroupPort.getRemoteProcessGroup().startTransmitting(remoteGroupPort);
-                    startedTransmitting++;
-                } catch (final Throwable t) {
-                    LOG.error("Unable to start transmitting with {} due to {}", new Object[]{remoteGroupPort, t});
-                }
-            }
-
-            LOG.info("Started {} Remote Group Ports transmitting", startedTransmitting);
-            startRemoteGroupPortsAfterInitialization.clear();
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    private ContentRepository createContentRepository(final NiFiProperties properties) throws InstantiationException, IllegalAccessException, ClassNotFoundException {
-        final String implementationClassName = properties.getProperty(NiFiProperties.CONTENT_REPOSITORY_IMPLEMENTATION, DEFAULT_CONTENT_REPO_IMPLEMENTATION);
-        if (implementationClassName == null) {
-            throw new RuntimeException("Cannot create Provenance Repository because the NiFi Properties is missing the following property: "
-                    + NiFiProperties.CONTENT_REPOSITORY_IMPLEMENTATION);
-        }
-
-        try {
-            final ContentRepository contentRepo = NarThreadContextClassLoader.createInstance(implementationClassName, ContentRepository.class);
-            synchronized (contentRepo) {
-                contentRepo.initialize(contentClaimManager);
-            }
-            return contentRepo;
-        } catch (final Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private ProvenanceEventRepository createProvenanceRepository(final NiFiProperties properties) throws InstantiationException, IllegalAccessException, ClassNotFoundException {
-        final String implementationClassName = properties.getProperty(NiFiProperties.PROVENANCE_REPO_IMPLEMENTATION_CLASS, DEFAULT_PROVENANCE_REPO_IMPLEMENTATION);
-        if (implementationClassName == null) {
-            throw new RuntimeException("Cannot create Provenance Repository because the NiFi Properties is missing the following property: "
-                    + NiFiProperties.PROVENANCE_REPO_IMPLEMENTATION_CLASS);
-        }
-
-        try {
-            return NarThreadContextClassLoader.createInstance(implementationClassName, ProvenanceEventRepository.class);
-        } catch (final Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private ComponentStatusRepository createComponentStatusRepository() {
-        final String implementationClassName = properties.getProperty(NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION, DEFAULT_COMPONENT_STATUS_REPO_IMPLEMENTATION);
-        if (implementationClassName == null) {
-            throw new RuntimeException("Cannot create Component Status Repository because the NiFi Properties is missing the following property: "
-                    + NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION);
-        }
-
-        try {
-            return NarThreadContextClassLoader.createInstance(implementationClassName, ComponentStatusRepository.class);
-        } catch (final Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * Creates a connection between two Connectable objects.
-     *
-     * @param id required ID of the connection
-     * @param name the name of the connection, or <code>null</code> to leave the
-     * connection unnamed
-     * @param source required source
-     * @param destination required destination
-     * @param relationshipNames required collection of relationship names
-     * @return
-     *
-     * @throws NullPointerException if the ID, source, destination, or set of
-     * relationships is null.
-     * @throws IllegalArgumentException if <code>relationships</code> is an
-     * empty collection
-     */
-    public Connection createConnection(final String id, final String name, final Connectable source, final Connectable destination, final Collection<String> relationshipNames) {
-        final StandardConnection.Builder builder = new StandardConnection.Builder(processScheduler);
-
-        final List<Relationship> relationships = new ArrayList<>();
-        for (final String relationshipName : requireNonNull(relationshipNames)) {
-            relationships.add(new Relationship.Builder().name(relationshipName).build());
-        }
-
-        return builder.id(requireNonNull(id).intern()).name(name == null ? null : name.intern()).relationships(relationships).source(requireNonNull(source)).destination(destination).build();
-    }
-
-    /**
-     * Creates a new Label
-     *
-     * @param id
-     * @param text
-     * @return
-     * @throws NullPointerException if either argument is null
-     */
-    public Label createLabel(final String id, final String text) {
-        return new StandardLabel(requireNonNull(id).intern(), text);
-    }
-
-    /**
-     * Creates a funnel
-     *
-     * @param id
-     * @return
-     */
-    public Funnel createFunnel(final String id) {
-        return new StandardFunnel(id.intern(), null, processScheduler);
-    }
-
-    /**
-     * Creates a Port to use as an Input Port for a Process Group
-     *
-     * @param id
-     * @param name
-     * @return
-     * @throws NullPointerException if the ID or name is not unique
-     * @throws IllegalStateException if an Input Port already exists with the
-     * same name or id.
-     */
-    public Port createLocalInputPort(String id, String name) {
-        id = requireNonNull(id).intern();
-        name = requireNonNull(name).intern();
-        verifyPortIdDoesNotExist(id);
-        return new LocalPort(id, name, null, ConnectableType.INPUT_PORT, processScheduler);
-    }
-
-    /**
-     * Creates a Port to use as an Output Port for a Process Group
-     *
-     * @param id
-     * @param name
-     * @return
-     * @throws NullPointerException if the ID or name is not unique
-     * @throws IllegalStateException if an Input Port already exists with the
-     * same name or id.
-     */
-    public Port createLocalOutputPort(String id, String name) {
-        id = requireNonNull(id).intern();
-        name = requireNonNull(name).intern();
-        verifyPortIdDoesNotExist(id);
-        return new LocalPort(id, name, null, ConnectableType.OUTPUT_PORT, processScheduler);
-    }
-
-    /**
-     * Creates a ProcessGroup with the given ID
-     *
-     * @param id
-     * @return
-     * @throws NullPointerException if the argument is null
-     */
-    public ProcessGroup createProcessGroup(final String id) {
-        return new StandardProcessGroup(requireNonNull(id).intern(), this, processScheduler, properties, encryptor);
-    }
-
-    /**
-     * <p>
-     * Creates a new ProcessorNode with the given type and identifier and initializes it invoking the
-     * methods annotated with {@link OnAdded}.
-     * </p>
-     *
-     * @param type
-     * @param id
-     * @return
-     * @throws NullPointerException if either arg is null
-     * @throws ProcessorInstantiationException if the processor cannot be
-     * instantiated for any reason
-     */
-    public ProcessorNode createProcessor(final String type, String id) throws ProcessorInstantiationException {
-        return createProcessor(type, id, true);
-    }
-    
-    /**
-     * <p>
-     * Creates a new ProcessorNode with the given type and identifier and optionally initializes it.
-     * </p>
-     *
-     * @param type the fully qualified Processor class name
-     * @param id the unique ID of the Processor
-     * @param firstTimeAdded whether or not this is the first time this Processor is added to the graph. If {@code true},
-     *                       will invoke methods annotated with the {@link OnAdded} annotation.
-     * @return
-     * @throws NullPointerException if either arg is null
-     * @throws ProcessorInstantiationException if the processor cannot be
-     * instantiated for any reason
-     */
-    public ProcessorNode createProcessor(final String type, String id, final boolean firstTimeAdded) throws ProcessorInstantiationException {
-        id = id.intern();
-        final Processor processor = instantiateProcessor(type, id);
-        final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider);
-        final ProcessorNode procNode = new StandardProcessorNode(processor, id, validationContextFactory, processScheduler, controllerServiceProvider);
-
-        final LogRepository logRepository = LogRepositoryFactory.getRepository(id);
-        logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, LogLevel.WARN, new ProcessorLogObserver(getBulletinRepository(), procNode));
-
-        if ( firstTimeAdded ) {
-            try (final NarCloseable x = NarCloseable.withNarLoader()) {
-                ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, processor);
-            } catch (final Exception e) {
-                logRepository.removeObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID);
-                throw new ProcessorLifeCycleException("Failed to invoke @OnAdded methods of " + procNode.getProcessor(), e);
-            }
-        }
-
-        return procNode;
-    }
-
-    private Processor instantiateProcessor(final String type, final String identifier) throws ProcessorInstantiationException {
-        Processor processor;
-
-        final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader();
-        try {
-            final ClassLoader detectedClassLoaderForType = ExtensionManager.getClassLoader(type);
-            final Class<?> rawClass;
-            if (detectedClassLoaderForType == null) {
-                // try to find from the current class loader
-                rawClass = Class.forName(type);
-            } else {
-                // try to find from the registered classloader for that type
-                rawClass = Class.forName(type, true, ExtensionManager.getClassLoader(type));
-            }
-
-            Thread.currentThread().setContextClassLoader(detectedClassLoaderForType);
-            final Class<? extends Processor> processorClass = rawClass.asSubclass(Processor.class);
-            processor = processorClass.newInstance();
-            final ProcessorLog processorLogger = new SimpleProcessLogger(identifier, processor);
-            final ProcessorInitializationContext ctx = new StandardProcessorInitializationContext(identifier, processorLogger, this);
-            processor.initialize(ctx);
-            return processor;
-        } catch (final Throwable t) {
-            throw new ProcessorInstantiationException(type, t);
-        } finally {
-            if (ctxClassLoader != null) {
-                Thread.currentThread().setContextClassLoader(ctxClassLoader);
-            }
-        }
-    }
-
-    /**
-     * @return the ExtensionManager used for instantiating Processors,
-     * Prioritizers, etc.
-     */
-    public ExtensionManager getExtensionManager() {
-        return extensionManager;
-    }
-
-    public String getInstanceId() {
-        readLock.lock();
-        try {
-            return instanceId;
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    /**
-     * Gets the BulletinRepository for storing and retrieving Bulletins.
-     *
-     * @return
-     */
-    public BulletinRepository getBulletinRepository() {
-        return bulletinRepository;
-    }
-
-    public SnippetManager getSnippetManager() {
-        return snippetManager;
-    }
-
-    /**
-     * Creates a Port to use as an Input Port for the root Process Group, which
-     * is used for Site-to-Site communications
-     *
-     * @param id
-     * @param name
-     * @return
-     * @throws NullPointerException if the ID or name is not unique
-     * @throws IllegalStateException if an Input Port already exists with the
-     * same name or id.
-     */
-    public Port createRemoteInputPort(String id, String name) {
-        id = requireNonNull(id).intern();
-        name = requireNonNull(name).intern();
-        verifyPortIdDoesNotExist(id);
-        return new StandardRootGroupPort(id, name, null, TransferDirection.RECEIVE, ConnectableType.INPUT_PORT, userService, getBulletinRepository(), processScheduler, Boolean.TRUE.equals(isSiteToSiteSecure));
-    }
-
-    /**
-     * Creates a Port to use as an Output Port for the root Process Group, which
-     * is used for Site-to-Site communications and will queue flow files waiting
-     * to be delivered to remote instances
-     *
-     * @param id
-     * @param name
-     * @return
-     * @throws NullPointerException if the ID or name is not unique
-     * @throws IllegalStateException if an Input Port already exists with the
-     * same name or id.
-     */
-    public Port createRemoteOutputPort(String id, String name) {
-        id = requireNonNull(id).intern();
-        name = requireNonNull(name).intern();
-        verifyPortIdDoesNotExist(id);
-        return new StandardRootGroupPort(id, name, null, TransferDirection.SEND, ConnectableType.OUTPUT_PORT, userService, getBulletinRepository(), processScheduler, Boolean.TRUE.equals(isSiteToSiteSecure));
-    }
-
-    /**
-     * Creates a new Remote Process Group with the given ID that points to the
-     * given URI
-     *
-     * @param id
-     * @param uri
-     * @return
-     *
-     * @throws NullPointerException if either argument is null
-     * @throws IllegalArgumentException if <code>uri</code> is not a valid URI.
-     */
-    public RemoteProcessGroup createRemoteProcessGroup(final String id, final String uri) {
-        return new StandardRemoteProcessGroup(requireNonNull(id).intern(), requireNonNull(uri).intern(), null, this, sslContext);
-    }
-
-    /**
-     * Verifies that no output port exists with the given id or name. If this
-     * does not hold true, throws an IllegalStateException
-     *
-     * @param id
-     * @throws IllegalStateException
-     */
-    private void verifyPortIdDoesNotExist(final String id) {
-        Port port = rootGroup.findOutputPort(id);
-        if (port != null) {
-            throw new IllegalStateException("An Input Port already exists with ID " + id);
-        }
-        port = rootGroup.findInputPort(id);
-        if (port != null) {
-            throw new IllegalStateException("An Input Port already exists with ID " + id);
-        }
-    }
-
-    /**
-     * @return the name of this controller, which is also the name of the Root
-     * Group.
-     */
-    public String getName() {
-        readLock.lock();
-        try {
-            return rootGroup.getName();
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    /**
-     * Sets the name for the Root Group, which also changes the name for the
-     * controller.
-     *
-     * @param name
-     */
-    public void setName(final String name) {
-        readLock.lock();
-        try {
-            rootGroup.setName(name);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    /**
-     * Gets the comments of this controller, which is also the comment of the
-     * Root Group.
-     *
-     * @return
-     */
-    public String getComments() {
-        readLock.lock();
-        try {
-            return rootGroup.getComments();
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    /**
-     * Sets the comment for the Root Group, which also changes the comment for
-     * the controller.
-     *
-     * @param comments
-     */
-    public void setComments(final String comments) {
-        readLock.lock();
-        try {
-            rootGroup.setComments(comments);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    /**
-     * @return <code>true</code> if the scheduling engine for this controller
-     * has been terminated.
-     */
-    public boolean isTerminated() {
-        this.readLock.lock();
-        try {
-            return (null == this.timerDrivenEngineRef.get() || this.timerDrivenEngineRef.get().isTerminated());
-        } finally {
-            this.readLock.unlock();
-        }
-    }
-
-    /**
-     * Triggers the controller to begin shutdown, stopping all processors and
-     * terminating the scheduling engine. After calling this method, the
-     * {@link #isTerminated()} method will indicate whether or not the shutdown
-     * has finished.
-     *
-     * @param kill if <code>true</code>, attempts to stop all active threads,
-     * but makes no guarantee that this will happen
-     *
-     * @throws IllegalStateException if the controller is already stopped or
-     * currently in the processor of stopping
-     */
-    public void shutdown(final boolean kill) {
-        this.shutdown = true;
-        stopAllProcessors();
-
-        writeLock.lock();
-        try {
-            if (isTerminated() || timerDrivenEngineRef.get().isTerminating()) {
-                throw new IllegalStateException("Controller already stopped or still stopping...");
-            }
-
-            if (kill) {
-                this.timerDrivenEngineRef.get().shutdownNow();
-                this.eventDrivenEngineRef.get().shutdownNow();
-                LOG.info("Initiated immediate shutdown of flow controller...");
-            } else {
-                this.timerDrivenEngineRef.get().shutdown();
-                this.eventDrivenEngineRef.get().shutdown();
-                LOG.info("Initiated graceful shutdown of flow controller...waiting up to " + gracefulShutdownSeconds + " seconds");
-            }
-
-            clusterTaskExecutor.shutdown();
-
-            // Trigger any processors' methods marked with @OnShutdown to be called
-            rootGroup.shutdown();
-
-            try {
-                this.timerDrivenEngineRef.get().awaitTermination(gracefulShutdownSeconds / 2, TimeUnit.SECONDS);
-                this.eventDrivenEngineRef.get().awaitTermination(gracefulShutdownSeconds / 2, TimeUnit.SECONDS);
-            } catch (final InterruptedException ie) {
-                LOG.info("Interrupted while waiting for controller termination.");
-            }
-
-            try {
-                flowFileRepository.close();
-            } catch (final Throwable t) {
-                LOG.warn("Unable to shut down FlowFileRepository due to {}", new Object[]{t});
-            }
-
-            if (this.timerDrivenEngineRef.get().isTerminated() && eventDrivenEngineRef.get().isTerminated()) {
-                LOG.info("Controller has been terminated successfully.");
-            } else {
-                LOG.warn("Controller hasn't terminated properly.  There exists an uninterruptable thread that will take an indeterminate amount of time to stop.  Might need to kill the program manually.");
-            }
-
-            if (externalSiteListener != null) {
-                externalSiteListener.stop();
-            }
-
-            if (flowFileSwapManager != null) {
-                flowFileSwapManager.shutdown();
-            }
-            
-            if ( processScheduler != null ) {
-            	processScheduler.shutdown();
-            }
-            
-            if ( contentRepository != null ) {
-                contentRepository.shutdown();
-            }
-            
-            if ( provenanceEventRepository != null ) {
-            	try {
-            		provenanceEventRepository.close();
-            	} catch (final IOException ioe) {
-            		LOG.warn("There was a problem shutting down the Provenance Repository: " + ioe.toString());
-            		if ( LOG.isDebugEnabled() ) {
-            			LOG.warn("", ioe);
-            		}
-            	}
-            }
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * Serializes the current state of the controller to the given OutputStream
-     *
-     * @param serializer
-     * @param os
-     * @throws FlowSerializationException if serialization of the flow fails for
-     * any reason
-     */
-    public void serialize(final FlowSerializer serializer, final OutputStream os) throws FlowSerializationException {
-        readLock.lock();
-        try {
-            serializer.serialize(this, os);
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    /**
-     * Synchronizes this controller with the proposed flow.
-     *
-     * For more details, see
-     * {@link FlowSynchronizer#sync(FlowController, DataFlow)}.
-     *
-     * @param synchronizer
-     * @param dataFlow the flow to load the controller with. If the flow is null
-     * or zero length, then the controller must not have a flow or else an
-     * UninheritableFlowException will be thrown.
-     *
-     * @throws FlowSerializationException if proposed flow is not a valid flow
-     * configuration file
-     * @throws UninheritableFlowException if the proposed flow cannot be loaded
-     * by the controller because in doing so would risk orphaning flow files
-     * @throws FlowSynchronizationException if updates to the controller failed.
-     * If this exception is thrown, then the controller should be considered
-     * unsafe to be used
-     */
-    public void synchronize(final FlowSynchronizer synchronizer, final DataFlow dataFlow)
-            throws FlowSerializationException, FlowSynchronizationException, UninheritableFlowException {
-        writeLock.lock();
-        try {
-            LOG.debug("Synchronizing controller with proposed flow");
-            synchronizer.sync(this, dataFlow, encryptor);
-            LOG.info("Successfully synchronized controller with proposed flow");
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * @return the currently configured maximum number of threads that can be
-     * used for executing processors at any given time.
-     */
-    public int getMaxTimerDrivenThreadCount() {
-        return maxTimerDrivenThreads.get();
-    }
-
-    public int getMaxEventDrivenThreadCount() {
-        return maxEventDrivenThreads.get();
-    }
-
-    public void setMaxTimerDrivenThreadCount(final int maxThreadCount) {
-        writeLock.lock();
-        try {
-            setMaxThreadCount(maxThreadCount, this.timerDrivenEngineRef.get(), this.maxTimerDrivenThreads);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    public void setMaxEventDrivenThreadCount(final int maxThreadCount) {
-        writeLock.lock();
-        try {
-            setMaxThreadCount(maxThreadCount, this.eventDrivenEngineRef.get(), this.maxEventDrivenThreads);
-            processScheduler.setMaxThreadCount(SchedulingStrategy.EVENT_DRIVEN, maxThreadCount);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * Updates the number of threads that can be simultaneously used for
-     * executing processors.
-     *
-     * @param maxThreadCount
-     *
-     * This method must be called while holding the write lock!
-     */
-    private void setMaxThreadCount(final int maxThreadCount, final FlowEngine engine, final AtomicInteger maxThreads) {
-        if (maxThreadCount < 1) {
-            throw new IllegalArgumentException();
-        }
-
-        maxThreads.getAndSet(maxThreadCount);
-        if (null != engine && engine.getCorePoolSize() < maxThreadCount) {
-            engine.setCorePoolSize(maxThreads.intValue());
-        }
-    }
-
-    /**
-     * @return the ID of the root group
-     */
-    public String getRootGroupId() {
-        readLock.lock();
-        try {
-            return rootGroup.getIdentifier();
-        } finally {
-            readLock.unlock();
-        }
-    }
-
-    /**
-     * Sets the root group to the given group
-     *
-     * @param group the ProcessGroup that is to become the new Root Group
-     *
-     * @throws IllegalArgumentException if the ProcessGroup has a parent
-     * @throws IllegalStateException if the FlowController does not know about
-     * the given process group
-     */
-    void setRootGroup(final ProcessGroup group) {
-        if (requireNonNull(group).getParent() != null) {
-            throw new IllegalArgumentException("A ProcessGroup that has a parent cannot be the Root Group");
-        }
-
-        writeLock.lock();
-        try {
-            rootGroup = group;
-
-            if (externalSiteListener != null) {
-                externalSiteListener.setRootGroup(group);
-            }
-
-            // update the heartbeat bean
-            this.heartbeatBeanRef.set(new HeartbeatBean(rootGroup, primary, connected));
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    public SystemDiagnostics getSystemDiagnostics() {
-        final SystemDiagnosticsFactory factory = new SystemDiagnosticsFactory();
-        return factory.create(flowFileRepository, contentRepository);
-    }
-
-    //
-    // ProcessGroup access
-    //
-    /**
-     * Updates the process group corresponding to the specified DTO. Any field
-     * in DTO that is <code>null</code> (with the exception of the required ID)
-     * will be ignored.
-     *
-     * @param dto
-     * @return a fully-populated DTO representing the newly updated ProcessGroup
-     * @throws ProcessorInstantiationException
-     *
-     * @throws IllegalStateException if no process group can be found with the
-     * ID of DTO or with the ID of the DTO's parentGroupId, if the template ID
-     * specified is invalid, or if the DTO's Parent Group ID changes but the
-     * parent group has incoming or outgoing connections
-     *
-     * @throws NullPointerException if the DTO or its ID is null
-     */
-    public void updateProcessGroup(final ProcessGroupDTO dto) throws ProcessorInstantiationException {
-        final ProcessGroup group = lookupGroup(requireNonNull(dto).getId());
-
-        final String name = dto.getName();
-        final PositionDTO position = dto.getPosition();
-        final String comments = dto.getComments();
-
-        if (name != null) {
-            group.setName(name);
-        }
-        if (position != null) {
-            group.setPosition(toPosition(position));
-        }
-        if (comments != null) {
-            group.setComments(comments);
-        }
-    }
-
-    //
-    // Template access
-    //
-    /**
-     * Adds a template to this controller. The contents of this template must be
-     * part of the current flow. This is going create a template based on a
-     * snippet of this flow.
-     *
-     * @param dto
-     * @return a copy of the given DTO
-     * @throws IOException if an I/O error occurs when persisting the Template
-     * @throws NullPointerException if the DTO is null
-     * @throws IllegalArgumentException if does not contain all required
-     * information, such as the template name or a processor's configuration
-     * element
-     */
-    public Template addTemplate(final TemplateDTO dto) throws IOException {
-        return templateManager.addTemplate(dto);
-    }
-
-    /**
-     * Removes all templates from this controller
-     *
-     * @throws IOException
-     */
-    public void clearTemplates() throws IOException {
-        templateManager.clear();
-    }
-
-    /**
-     * Imports the specified template into this controller. The contents of this
-     * template may have come from another NiFi instance.
-     *
-     * @param dto
-     * @return
-     * @throws IOException
-     */
-    public Template importTemplate(final TemplateDTO dto) throws IOException {
-        return templateManager.importTemplate(dto);
-    }
-
-    /**
-     * Returns the template with the given ID, or <code>null</code> if no
-     * template exists with the given ID.
-     *
-     * @param id
-     * @return
-     */
-    public Template getTemplate(final String id) {
-        return templateManager.getTemplate(id);
-    }
-
-    public TemplateManager getTemplateManager() {
-        return templateManager;
-    }
-
-    /**
-     * Returns all templates that this controller knows about.
-     *
-     * @return
-     */
-    public Collection<Template> getTemplates() {
-        return templateManager.getTemplates();
-    }
-
-    /**
-     * Removes the template with the given ID.
-     *
-     * @param id the ID of the template to remove
-     * @throws NullPointerException if the argument is null
-     * @throws IllegalStateException if no template exists with the given ID
-     * @throws IOException if template could not be removed
-     */
-    public void removeTemplate(final String id) throws IOException, IllegalStateException {
-        templateManager.removeTemplate(id);
-    }
-
-    private Position toPosition(final PositionDTO dto) {
-        return new Position(dto.getX(), dto.getY());
-    }
-
-    //
-    // Snippet
-    //
-    /**
-     * Creates an instance of the given snippet and adds the components to the
-     * given group
-     *
-     * @param group
-     * @param dto
-     *
-     * @throws NullPointerException if either argument is null
-     * @throws IllegalStateException if the snippet is not valid because a
-     * component in the snippet has an ID that is not unique to this flow, or
-     * because it shares an Input Port or Output Port at the root level whose
-     * name already exists in the given ProcessGroup, or because the Template
-     * contains a Processor or a Prioritizer whose class is not valid within
-     * this instance of NiFi.
-     * @throws ProcessorInstantiationException if unable to instantiate a
-     * processor
-     */
-    public void instantiateSnippet(final ProcessGroup group, final FlowSnippetDTO dto) throws ProcessorInstantiationException {
-        writeLock.lock();
-        try {
-            validateSnippetContents(requireNonNull(group), dto);
-
-            //
-            // Instantiate the labels
-            //
-            for (final LabelDTO labelDTO : dto.getLabels()) {
-                final Label label = createLabel(labelDTO.getId(), labelDTO.getLabel());
-                label.setPosition(toPosition(labelDTO.getPosition()));
-                if (labelDTO.getWidth() != null && labelDTO.getHeight() != null) {
-                    label.setSize(new Size(labelDTO.getWidth(), labelDTO.getHeight()));
-                }
-
-                // TODO: Update the label's "style"
-                group.addLabel(label);
-            }
-
-            // 
-            // Instantiate the funnels
-            for (final FunnelDTO funnelDTO : dto.getFunnels()) {
-                final Funnel funnel = createFunnel(funnelDTO.getId());
-                funnel.setPosition(toPosition(funnelDTO.getPosition()));
-                group.addFunnel(funnel);
-            }
-
-            //
-            // Instantiate Input Ports & Output Ports
-            //
-            for (final PortDTO portDTO : dto.getInputPorts()) {
-                final Port inputPort;
-                if (group.isRootGroup()) {
-                    inputPort = createRemoteInputPort(portDTO.getId(), portDTO.getName());
-                    inputPort.setMaxConcurrentTasks(portDTO.getConcurrentlySchedulableTaskCount());
-                    if (portDTO.getGroupAccessControl() != null) {
-                        ((RootGroupPort) inputPort).setGroupAccessControl(portDTO.getGroupAccessControl());
-                    }
-                    if (portDTO.getUserAccessControl() != null) {
-                        ((RootGroupPort) inputPort).setUserAccessControl(portDTO.getUserAccessControl());
-                    }
-                } else {
-                    inputPort = createLocalInputPort(portDTO.getId(), portDTO.getName());
-                }
-
-                inputPort.setPosition(toPosition(portDTO.getPosition()));
-                inputPort.setProcessGroup(group);
-                inputPort.setComments(portDTO.getComments());
-                group.addInputPort(inputPort);
-            }
-
-            for (final PortDTO portDTO : dto.getOutputPorts()) {
-                final Port outputPort;
-                if (group.isRootGroup()) {
-                    outputPort = createRemoteOutputPort(portDTO.getId(), portDTO.getName());
-                    outputPort.setMaxConcurrentTasks(portDTO.getConcurrentlySchedulableTaskCount());
-                    if (portDTO.getGroupAccessControl() != null) {
-                        ((RootGroupPort) outputPort).setGroupAccessControl(portDTO.getGroupAccessControl());
-                    }
-                    if (portDTO.getUserAccessControl() != null) {
-                        ((RootGroupPort) outputPort).setUserAccessControl(portDTO.getUserAccessControl());
-                    }
-                } else {
-                    outputPort = createLocalOutputPort(portDTO.getId(), portDTO.getName());
-                }
-
-                outputPort.setPosition(toPosition(portDTO.getPosition()));
-                outputPort.setProcessGroup(group);
-                outputPort.setComments(portDTO.getComments());
-                group.addOutputPort(outputPort);
-            }
-
-            //
-            // Instantiate the processors
-            //
-            for (final ProcessorDTO processorDTO : dto.getProcessors()) {
-                final ProcessorNode procNode = createProcessor(processorDTO.getType(), processorDTO.getId());
-
-                procNode.setPosition(toPosition(processorDTO.getPosition()));
-                procNode.setProcessGroup(group);
-
-                final ProcessorConfigDTO config = processorDTO.getConfig();
-                procNode.setComments(config.getComments());
-                if (config.isLossTolerant() != null) {
-                    procNode.setLossTolerant(config.isLossTolerant());
-                }
-                procNode.setName(processorDTO.getName());
-
-                procNode.setYieldPeriod(config.getYieldDuration());
-                procNode.setPenalizationPeriod(config.getPenaltyDuration());
-                procNode.setBulletinLevel(LogLevel.valueOf(config.getBulletinLevel()));
-                procNode.setAnnotationData(config.getAnnotationData());
-                procNode.setStyle(processorDTO.getStyle());
-
-                if (config.getRunDurationMillis() != null) {
-                    procNode.setRunDuration(config.getRunDurationMillis(), TimeUnit.MILLISECONDS);
-                }
-
-                if (config.getSchedulingStrategy() != null) {
-                    procNode.setSchedulingStrategy(SchedulingStrategy.valueOf(config.getSchedulingStrategy()));
-                }
-
-                // ensure that the scheduling strategy is set prior to these values
-                procNode.setMaxConcurrentTasks(config.getConcurrentlySchedulableTaskCount());
-                procNode.setScheduldingPeriod(config.getSchedulingPeriod());
-
-                final Set<Relationship> relationships = new HashSet<>();
-                if (processorDTO.getRelationships() != null) {
-                    for (final RelationshipDTO rel : processorDTO.getRelationships()) {
-                        if (rel.isAutoTerminate()) {
-                            relationships.add(procNode.getRelationship(rel.getName()));
-                        }
-                    }
-                    procNode.setAutoTerminatedRelationships(relationships);
-                }
-
-                if (config.getProperties() != null) {
-                    for (Map.Entry<String, String> entry : config.getProperties().entrySet()) {
-                        if (entry.getValue() != null) {
-                            procNode.setProperty(entry.getKey(), entry.getValue());
-                        }
-                    }
-                }
-
-                group.addProcessor(procNode);
-            }
-
-            //
-            // Instantiate Remote Process Groups
-            //
-            for (final RemoteProcessGroupDTO remoteGroupDTO : dto.getRemoteProcessGroups()) {
-                final RemoteProcessGroup remoteGroup = createRemoteProcessGroup(remoteGroupDTO.getId(), remoteGroupDTO.getTargetUri());
-                remoteGroup.setComments(remoteGroupDTO.getComments());
-                remoteGroup.setPosition(toPosition(remoteGroupDTO.getPosition()));
-                remoteGroup.setCommunicationsTimeout(remoteGroupDTO.getCommunicationsTimeout());
-                remoteGroup.setYieldDuration(remoteGroupDTO.getYieldDuration());
-                remoteGroup.setProcessGroup(group);
-
-                // set the input/output ports
-                if (remoteGroupDTO.getContents() != null) {
-                    final RemoteProcessGroupContentsDTO contents = remoteGroupDTO.getContents();
-
-                    // ensure there input ports
-                    if (contents.getInputPorts() != null) {
-                        remoteGroup.setInputPorts(convertRemotePort(contents.getInputPorts()));
-                    }
-
-                    // ensure there are output ports
-                    if (contents.getOutputPorts() != null) {
-                        remoteGroup.setOutputPorts(convertRemotePort(contents.getOutputPorts()));
-                    }
-                }
-
-                group.addRemoteProcessGroup(remoteGroup);
-            }
-
-            // 
-            // Instantiate ProcessGroups
-            //
-            for (final ProcessGroupDTO groupDTO : dto.getProcessGroups()) {
-                final ProcessGroup childGroup = createProcessGroup(groupDTO.getId());
-                childGroup.setParent(group);
-                childGroup.setPosition(toPosition(groupDTO.getPosition()));
-                childGroup.setComments(groupDTO.getComments());
-                childGroup.setName(groupDTO.getName());
-                group.addProcessGroup(childGroup);
-
-                final FlowSnippetDTO contents = groupDTO.getContents();
-
-                // we want this to be recursive, so we will create a new template that contains only
-                // the contents of this child group and recursively call ourselves.
-                final FlowSnippetDTO childTemplateDTO = new FlowSnippetDTO();
-                childTemplateDTO.setConnections(contents.getConnections());
-                childTemplateDTO.setInputPorts(contents.getInputPorts());
-                childTemplateDTO.setLabels(contents.getLabels());
-                childTemplateDTO.setOutputPorts(contents.getOutputPorts());
-                childTemplateDTO.setProcessGroups(contents.getProcessGroups());
-                childTemplateDTO.setProcessors(contents.getProcessors());
-                childTemplateDTO.setFunnels(contents.getFunnels());
-                childTemplateDTO.setRemoteProcessGroups(contents.getRemoteProcessGroups());
-                instantiateSnippet(childGroup, childTemplateDTO);
-            }
-
-            //
-            // Instantiate Connections
-            //
-            for (final ConnectionDTO connectionDTO : dto.getConnections()) {
-                final ConnectableDTO sourceDTO = connectionDTO.getSource();
-                final ConnectableDTO destinationDTO = connectionDTO.getDestination();
-                final Connectable source;
-                final Connectable destination;
-
-                // locate the source and destination connectable. if this is a remote port 
-                // we need to locate the remote process groups. otherwise we need to 
-                // find the connectable given its parent group.
-                // NOTE: (getConnectable returns ANY connectable, when the parent is
-                // not this group only input ports or output ports should be returned. if something 
-                // other than a port is returned, an exception will be thrown when adding the 
-                // connection below.)
-                // see if the source connectable is a remote port
-                if (ConnectableType.REMOTE_OUTPUT_PORT.name().equals(sourceDTO.getType())) {
-                    final RemoteProcessGroup remoteGroup = group.getRemoteProcessGroup(sourceDTO.getGroupId());
-                    source = remoteGroup.getOutputPort(sourceDTO.getId());
-                } else {
-                    final ProcessGroup sourceGroup = getConnectableParent(group, sourceDTO.getGroupId());
-                    source = sourceGroup.getConnectable(sourceDTO.getId());
-                }
-
-                // see if the destination connectable is a remote port
-                if (ConnectableType.REMOTE_INPUT_PORT.name().equals(destinationDTO.getType())) {
-                    final RemoteProcessGroup remoteGroup = group.getRemoteProcessGroup(destinationDTO.getGroupId());
-                    destination = remoteGroup.getInputPort(destinationDTO.getId());
-                } else {
-                    final ProcessGroup destinationGroup = getConnectableParent(group, destinationDTO.getGroupId());
-                    destination = destinationGroup.getConnectable(destinationDTO.getId());
-                }
-
-                // determine the selection relationships for this connection
-                final Set<String> relationships = new HashSet<>();
-                if (connectionDTO.getSelectedRelationships() != null) {
-                    relationships.addAll(connectionDTO.getSelectedRelationships());
-                }
-
-                final Connection connection = createConnection(connectionDTO.getId(), connectionDTO.getName(), source, destination, relationships);
-
-                if (connectionDTO.getBends() != null) {
-                    final List<Position> bendPoints = new ArrayList<>();
-                    for (final PositionDTO bend : connectionDTO.getBends()) {
-                        bendPoints.add(new Position(bend.getX(), bend.getY()));
-                    }
-                    connection.setBendPoints(bendPoints);
-                }
-
-                final FlowFileQueue queue = connection.getFlowFileQueue();
-                queue.setBackPressureDataSizeThreshold(connectionDTO.getBackPressureDataSizeThreshold());
-                queue.setBackPressureObjectThreshold(connectionDTO.getBackPressureObjectThreshold());
-                queue.setFlowFileExpiration(connectionDTO.getFlowFileExpiration());
-
-                final List<String> prioritizers = connectionDTO.getPrioritizers();
-                if (prioritizers != null) {
-                    final List<String> newPrioritizersClasses = new ArrayList<>(prioritizers);
-                    final List<FlowFilePrioritizer> newPrioritizers = new ArrayList<>();
-                    for (final String className : newPrioritizersClasses) {
-                        try {
-                            newPrioritizers.add(createPrioritizer(className));
-                        } catch (final ClassNotFoundException | InstantiationException | IllegalAccessException e) {
-                            throw new IllegalArgumentException("Unable to set prioritizer " + className + ": " + e);
-                        }
-                    }
-                    queue.setPriorities(newPrioritizers);
-                }
-
-                connection.setProcessGroup(group);
-                group.addConnection(connection);
-            }
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    /**
-     * Converts a set of ports into a set of remote process group ports.
-     *
-     * @param ports
-     * @return
-     */
-    private Set<RemoteProcessGroupPortDescriptor> convertRemotePort(final Set<RemoteProcessGroupPortDTO> ports) {
-        Set<RemoteProcessGroupPortDescriptor> remotePorts = null;
-        if (ports != null) {
-            remotePorts = new LinkedHashSet<>(ports.size());
-            for (RemoteProcessGroupPortDTO port : ports) {
-                final StandardRemoteProcessGroupPortDescriptor descriptor = new StandardRemoteProcessGroupPortDescriptor();
-                descriptor.setId(port.getId());
-                descriptor.setName(port.getName());
-                descriptor.setComments(port.getComments());
-                descriptor.setTargetRunning(port.isTargetRunning());
-                descriptor.setConnected(port.isConnected());
-                descriptor.setConcurrentlySchedulableTaskCount(port.getConcurrentlySchedulableTaskCount());
-                descriptor.setTransmitting(port.isTransmitting());
-                descriptor.setUseCompression(port.getUseCompression());
-                remotePorts.add(descriptor);
-            }
-        }
-        return remotePorts;
-    }
-
-    /**
-     * Returns the parent of the specified Connectable. This only considers this
-     * group and any direct child sub groups.
-     *
-     * @param parentGroupId
-     * @return
-     */
-    private ProcessGroup getConnectableParent(final ProcessGroup group, final String parentGroupId) {
-        if (areGroupsSame(group.getIdentifier(), parentGroupId)) {
-            return group;
-        } else {
-            return group.getProcessGroup(parentGroupId);
-        }
-    }
-
-    /**
-     * <p>
-     * Verifies that the given DTO is valid, according to the following:
-     *
-     * <ul>
-     * <li>None of the ID's in any component of the DTO can be used in this
-     * flow.</li>
-     * <li>The ProcessGroup to which the template's contents will be added must
-     * not contain any InputPort or OutputPort with the same name as one of the
-     * corresponding components in the root level of the template.</li>
-     * <li>All Processors' classes must exist in this instance.</li>
-     * <li>All Flow File Prioritizers' classes must exist in this instance.</li>
-     * </ul>
-     * </p>
-     *
-     * <p>
-     * If any of the above statements does not hold true, an
-     * {@link IllegalStateException} or a
-     * {@link ProcessorInstantiationException} will be thrown.
-     * </p>
-     *
-     * @param group
-     * @param templateContents
-     */
-    private void validateSnippetContents(final ProcessGroup group, final FlowSnippetDTO templateContents) {
-        // validate the names of Input Ports
-        for (final PortDTO port : templateContents.getInputPorts()) {
-            if (group.getInputPortByName(port.getName()) != null) {
-                throw new IllegalStateException("ProcessGroup already has an Input Port with name " + port.getName());
-            }
-        }
-
-        // validate the names of Output Ports
-        for (final PortDTO port : templateContents.getOutputPorts()) {
-            if (group.getOutputPortByName(port.getName()) != null) {
-                throw new IllegalStateException("ProcessGroup already has an Output Port with name " + port.getName());
-            }
-        }
-
-        // validate that all Processor Types and Prioritizer Types are valid
-        final List<String> processorClasses = new ArrayList<>();
-        for (final Class<?> c : ExtensionManager.getExtensions(Processor.class)) {
-            processorClasses.add(c.getName());
-        }
-        final List<String> prioritizerClasses = new ArrayList<>();
-        for (final Class<?> c : ExtensionManager.getExtensions(FlowFilePrioritizer.class)) {
-            prioritizerClasses.add(c.getName());
-        }
-
-        final Set<ProcessorDTO> allProcs = new HashSet<>();
-        final Set<ConnectionDTO> allConns = new HashSet<>();
-        allProcs.addAll(templateContents.getProcessors());
-        allConns.addAll(templateContents.getConnections());
-        for (final ProcessGroupDTO childGroup : templateContents.getProcessGroups()) {
-            allProcs.addAll(findAllProcessors(childGroup));
-            allConns.addAll(findAllConnections(childGroup));
-        }
-
-        for (final ProcessorDTO proc : allProcs) {
-            if (!processorClasses.contains(proc.getType())) {
-                throw new IllegalStateException("Invalid Processor Type: " + proc.getType());
-            }
-        }
-
-        for (final ConnectionDTO conn : allConns) {
-            final List<String> prioritizers = conn.getPrioritizers();
-            if (prioritizers != null) {
-                for (final String prioritizer : prioritizers) {
-                    if (!prioritizerClasses.contains(prioritizer)) {
-                        throw new IllegalStateException("Invalid FlowFile Prioritizer Type: " + prioritizer);
-                    }
-                }
-            }
-        }
-    }
-
-    /**
-     * Recursively finds all ProcessorDTO's
-     *
-     * @param group
-     * @return
-     */
-    private Set<ProcessorDTO> findAllProcessors(final ProcessGroupDTO group) {
-        final Set<ProcessorDTO> procs = new HashSet<>();
-        for (final ProcessorDTO dto : group.getContents().getProcessors()) {
-            procs.add(dto);
-        }
-
-        for (final ProcessGroupDTO childGroup : group.getContents().getProcessGroups()) {
-            procs.addAll(findAllProcessors(childGroup));
-        }
-        return procs;
-    }
-
-    /**
-     * Recursively finds all ConnectionDTO's
-     *
-     * @param group
-     * @return
-     */
-    private Set<ConnectionDTO> findAllConnections(final ProcessGroupDTO group) {
-        final Set<ConnectionDTO> conns = new HashSet<>();
-        for (final ConnectionDTO dto : group.getContents().getConnections()) {
-            conns.add(dto);
-        }
-
-        for (final ProcessGroupDTO childGroup : group.getContents().getProcessGroups()) {
-            conns.addAll(findAllConnections(childGroup));
-        }
-        return conns;
-    }
-
-    //
-    // Processor access
-    //
-    /**
-     * Indicates whether or not the two ID's point to the same ProcessGroup. If
-     * either id is null, will return <code>false</code.
-     *
-     * @param id1
-     * @param id2
-     * @return
-     */
-    public boolean areGroupsSame(final String id1, final String id2) {
-        if (id1 == null || id2 == null) {
-            return false;
-        } else if (id1.equals(id2)) {
-            return true;
-        } else {
-            final String comparable1 = (id1.equals(ROOT_GROUP_ID_ALIAS) ? getRootGroupId() : id1);
-            final String comparable2 = (id2.equals(ROOT_GROUP_ID_ALIAS) ? getRootGroupId() : id2);
-            return (comparable1.equals(comparable2));
-        }
-    }
-
-    public FlowFilePrioritizer createPrioritizer(final String type) throws InstantiationException, IllegalAccessException, ClassNotFoundException {
-        FlowFilePrioritizer prioritizer;
-
-        final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader();
-        try {
-            final ClassLoader detectedClassLoaderForType = ExtensionManager.getClassLoader(type);
-            final Class<?> rawClass;
-            if (detectedClassLoaderForType == null) {
-                // try to find from the current class loader
-                rawClass = Class.forName(type);
-            } else {
-                // try to find from the registered classloader for that type
-                rawClass = Class.forName(type, true, ExtensionManager.getClassLoader(type));
-            }
-
-            Thread.currentThread().setContextClassLoader(detectedClassLoaderForType);
-            final Class<? extends FlowFilePrioritizer> prioritizerClass = rawClass.asSubclass(FlowFilePrioritizer.class);
-            final Object processorObj = prioritizerClass.newInstance();
-            prioritizer = prioritizerClass.cast(processorObj);
-
-            return prioritizer;
-        } finally {
-            if (ctxClassLoader != null) {
-                Thread.currentThread().setContextClassLoader(ctxClassLoader);
-            }
-        }
-    }
-
-    //
-    // InputPort access
-    //
-    public PortDTO updateInputPort(final String parentGroupId, final PortDTO dto) {
-        final ProcessGroup parentGroup = lookupGroup(parentGroupId);
-        final Port port = parentGroup.getInputPort(dto.getId());
-        if (port == null) {
-            throw new IllegalStateException("No Input Port with ID " + dto.getId() + " is known as a child of ProcessGroup with ID " + parentGroupId);
-        }
-
-        final String name = dto.getName();
-        if (dto.getPosition() != null) {
-            port.setPosition(toPosition(dto.getPosition()));
-        }
-
-        if (name != null) {
-            port.setName(name);
-        }
-
-        return createDTO(port);
-    }
-
-    private PortDTO createDTO(final Port port) {
-        if (port == null) {
-            return null;
-        }
-
-        final PortDTO dto = new PortDTO();
-        dto.setId(port.getIdentifier());
-        dto.setPosition(new PositionDTO(port.getPosition().getX(), port.getPosition().getY()));
-        dto.setName(port.getName());
-        dto.setParentGroupId(port.getProcessGroup().getIdentifier());
-
-        return dto;
-    }
-
-    //
-    // OutputPort access
-    //
-    public PortDTO updateOutputPort(final String parentGroupId, final PortDTO dto) {
-        final ProcessGroup parentGroup = lookupGroup(parentGroupId);
-        final Port port = parentGroup.getOutputPort(dto.getId());
-        if (port == null) {
-            throw new IllegalStateException("No Output Port with ID " + dto.getId() + " is known as a child of ProcessGroup with ID " + parentGroupId);
-        }
-
-        final String name = dto.getName();
-        if (name != null) {
-            port.setName(name);
-        }
-
-        if (dto.getPosition() != null) {
-            port.setPosition(toPosition(dto.getPosition()));
-        }
-
-        return createDTO(port);
-    }
-
-    //
-    // Processor/Prioritizer/Filter Class Access
-    //
-    @SuppressWarnings("rawtypes")
-    public Set<Class> getFlowFileProcessorClasses() {
-        return ExtensionManager.getExtensions(Processor.class);
-    }
-
-    @SuppressWarnings("rawtypes")
-    public Set<Class> getFlowFileComparatorClasses() {
-        return ExtensionManager.getExtensions(FlowFilePrioritizer.class);
-    }
-
-    /**
-     * Returns the ProcessGroup with the given ID
-     *
-     * @param id
-     * @return the process group or null if not group is found
-     */
-    private ProcessGroup lookupGroup(final String id) {
-        final ProcessGroup group = getGroup(id);
-        if (group == null) {
-            throw new IllegalStateException("No Group with ID " + id + " exists");
-        }
-        return group;
-    }
-
-    /**
-     * Returns the ProcessGroup with the given ID
-     *
-     * @param id
-     * @return the process group or null if not group is found
-     */
-    public ProcessGroup getGroup(final String id) {
-        requireNonNull(id);
-        final ProcessGroup root;
-        readLock.lock();
-        try {
-            root = rootGroup;
-        } finally {
-            readLock.unlock();
-        }
-
-        final String searchId = id.equals(ROOT_GROUP_ID_ALIAS) ? getRootGroupId() : id;
-        return (root == null) ? null : root.findProcessGroup(searchId);
-    }
-
-    @Override
-    public ProcessGroupStatus getControllerStatus() {
-        return getGroupStatus(getRootGroupId());
-    }
-
-    public ProcessGroupStatus getGroupStatus(final String groupId) {
-        return getGroupStatus(groupId, getProcessorStats());
-    }
-
-    public ProcessGroupStatus getGroupStatus(final String groupId, final RepositoryStatusReport statusReport) {
-        final ProcessGroup group = getGroup(groupId);
-        return getGroupStatus(group, statusReport);
-    }
-
-    public ProcessGroupStatus getGroupStatus(final ProcessGroup group, final RepositoryStatusReport statusReport) {
-        if (group == null) {
-            return null;
-        }
-
-        final ProcessGroupStatus status = new ProcessGroupStatus();
-        status.setId(group.getIdentifier());
-        status.setName(group.getName());
-        status.setCreationTimestamp(new Date().getTime());
-        int activeGroupThreads = 0;
-        long bytesRead = 0L;
-        long bytesWritten = 0L;
-        int queuedCount = 0;
-        long queuedContentSize = 0L;
-        int flowFilesIn = 0;
-        long bytesIn = 0L;
-        int flowFilesOut = 0;
-        long bytesOut = 0L;
-        int flowFilesReceived = 0;
-        long bytesReceived = 0L;
-        int flowFilesSent = 0;
-        long bytesSent = 0L;
-
-        // set status for processors
-        final Collection<ProcessorStatus> processorStatusCollection = new ArrayList<>();
-        status.setProcessorStatus(processorStatusCollection);
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            final ProcessorStatus procStat = getProcessorStatus(statusReport, procNode);
-            processorStatusCollection.add(procStat);
-            activeGroupThreads += procStat.getActiveThreadCount();
-            bytesRead += procStat.getBytesRead();
-            bytesWritten += procStat.getBytesWritten();
-
-            flowFilesReceived += procStat.getFlowFilesReceived();
-            bytesReceived += procStat.getBytesReceived();
-            flowFilesSent += procStat.getFlowFilesSent();
-            bytesSent += procStat.getBytesSent();
-        }
-
-        // set status for local child groups     
-        final Collection<ProcessGroupStatus> localChildGroupStatusCollection = new ArrayList<>();
-        status.setProcessGroupStatus(localChildGroupStatusCollection);
-        for (final ProcessGroup childGroup : group.getProcessGroups()) {
-            final ProcessGroupStatus childGroupStatus = getGroupStatus(childGroup, statusReport);
-            localChildGroupStatusCollection.add(childGroupStatus);
-            activeGroupThreads += childGroupStatus.getActiveThreadCount();
-            bytesRead += childGroupStatus.getBytesRead();
-            bytesWritten += childGroupStatus.getBytesWritten();
-            queuedCount += childGroupStatus.getQueuedCount();
-            queuedContentSize += childGroupStatus.getQueuedContentSize();
-
-            flowFilesReceived += childGroupStatus.getFlowFilesReceived();
-            bytesReceived += childGroupStatus.getBytesReceived();
-            flowFilesSent += childGroupStatus.getFlowFilesSent();
-            bytesSent += childGroupStatus.getBytesSent();
-        }
-
-        // set status for remote child groups
-        final Collection<RemoteProcessGroupStatus> remoteProcessGroupStatusCollection = new ArrayList<>();
-        status.setRemoteProcessGroupStatus(remoteProcessGroupStatusCollection);
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            final RemoteProcessGroupStatus remoteStatus = createRemoteGroupStatus(remoteGroup, statusReport);
-            if (remoteStatus != null) {
-                remoteProcessGroupStatusCollection.add(remoteStatus);
-
-                flowFilesReceived += remoteStatus.getReceivedCount();
-                bytesReceived += remoteStatus.getReceivedContentSize();
-                flowFilesSent += remoteStatus.getSentCo

<TRUNCATED>

[27/51] [partial] incubator-nifi git commit: Reworked overall directory structure to make releasing nifi vs maven plugis easier

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/AuthorityProviderFactoryBean.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/AuthorityProviderFactoryBean.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/AuthorityProviderFactoryBean.java
deleted file mode 100644
index 31a01be..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/AuthorityProviderFactoryBean.java
+++ /dev/null
@@ -1,529 +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.authorization;
-
-import java.io.File;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.Field;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import javax.xml.XMLConstants;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBElement;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.transform.stream.StreamSource;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import org.apache.nifi.authorization.annotation.AuthorityProviderContext;
-import org.apache.nifi.authorization.exception.AuthorityAccessException;
-import org.apache.nifi.authorization.exception.IdentityAlreadyExistsException;
-import org.apache.nifi.authorization.exception.ProviderCreationException;
-import org.apache.nifi.authorization.exception.ProviderDestructionException;
-import org.apache.nifi.authorization.exception.UnknownIdentityException;
-import org.apache.nifi.authorization.generated.AuthorityProviders;
-import org.apache.nifi.authorization.generated.Property;
-import org.apache.nifi.authorization.generated.Provider;
-import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.nar.NarCloseable;
-import org.apache.nifi.util.NiFiProperties;
-import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.beans.BeansException;
-import org.springframework.beans.factory.DisposableBean;
-import org.springframework.beans.factory.FactoryBean;
-import org.springframework.context.ApplicationContext;
-import org.springframework.context.ApplicationContextAware;
-import org.xml.sax.SAXException;
-
-/**
- * Factory bean for loading the configured authority provider.
- */
-public class AuthorityProviderFactoryBean implements FactoryBean, ApplicationContextAware, DisposableBean, AuthorityProviderLookup {
-
-    private static final Logger logger = LoggerFactory.getLogger(AuthorityProviderFactoryBean.class);
-    private static final String AUTHORITY_PROVIDERS_XSD = "/authority-providers.xsd";
-    private static final String JAXB_GENERATED_PATH = "org.apache.nifi.authorization.generated";
-    private static final JAXBContext JAXB_CONTEXT = initializeJaxbContext();
-
-    /**
-     * Load the JAXBContext.
-     */
-    private static JAXBContext initializeJaxbContext() {
-        try {
-            return JAXBContext.newInstance(JAXB_GENERATED_PATH, AuthorityProviderFactoryBean.class.getClassLoader());
-        } catch (JAXBException e) {
-            throw new RuntimeException("Unable to create JAXBContext.");
-        }
-    }
-
-    private ApplicationContext applicationContext;
-    private AuthorityProvider authorityProvider;
-    private NiFiProperties properties;
-    private final Map<String, AuthorityProvider> authorityProviders = new HashMap<>();
-
-    @Override
-    public AuthorityProvider getAuthorityProvider(String identifier) {
-        return authorityProviders.get(identifier);
-    }
-
-    @Override
-    public Object getObject() throws Exception {
-        if (authorityProvider == null) {
-            // look up the authority provider to use
-            final String authorityProviderIdentifier = properties.getProperty(NiFiProperties.SECURITY_USER_AUTHORITY_PROVIDER);
-
-            // ensure the authority provider class name was specified
-            if (StringUtils.isBlank(authorityProviderIdentifier)) {
-                // if configured for ssl, the authority provider must be specified
-                if (properties.getSslPort() != null) {
-                    throw new Exception("When running securely, the authority provider identifier must be specified in the nifi properties file.");
-                }
-
-                // use a default provider... only allowable when running not securely
-                authorityProvider = createDefaultProvider();
-            } else {
-                final AuthorityProviders authorityProviderConfiguration = loadAuthorityProvidersConfiguration();
-
-                // create each authority provider
-                for (final Provider provider : authorityProviderConfiguration.getProvider()) {
-                    authorityProviders.put(provider.getIdentifier(), createAuthorityProvider(provider.getIdentifier(), provider.getClazz()));
-                }
-
-                // configure each authority provider
-                for (final Provider provider : authorityProviderConfiguration.getProvider()) {
-                    final AuthorityProvider instance = authorityProviders.get(provider.getIdentifier());
-                    instance.onConfigured(loadAuthorityProviderConfiguration(provider));
-                }
-
-                // get the authority provider instance
-                authorityProvider = getAuthorityProvider(authorityProviderIdentifier);
-
-                // ensure it was found
-                if (authorityProvider == null) {
-                    throw new Exception(String.format("The specified authority provider '%s' could not be found.", authorityProviderIdentifier));
-                }
-            }
-        }
-
-        return authorityProvider;
-    }
-
-    /**
-     * Loads the authority providers configuration.
-     *
-     * @return
-     * @throws Exception
-     */
-    private AuthorityProviders loadAuthorityProvidersConfiguration() throws Exception {
-        final File authorityProvidersConfigurationFile = properties.getAuthorityProviderConfiguraitonFile();
-
-        // load the users from the specified file
-        if (authorityProvidersConfigurationFile.exists()) {
-            try {
-                // find the schema
-                final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
-                final Schema schema = schemaFactory.newSchema(AuthorityProviders.class.getResource(AUTHORITY_PROVIDERS_XSD));
-
-                // attempt to unmarshal
-                final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
-                unmarshaller.setSchema(schema);
-                final JAXBElement<AuthorityProviders> element = unmarshaller.unmarshal(new StreamSource(authorityProvidersConfigurationFile), AuthorityProviders.class);
-                return element.getValue();
-            } catch (SAXException | JAXBException e) {
-                throw new Exception("Unable to load the authority provider configuration file at: " + authorityProvidersConfigurationFile.getAbsolutePath());
-            }
-        } else {
-            throw new Exception("Unable to find the authority provider configuration file at " + authorityProvidersConfigurationFile.getAbsolutePath());
-        }
-    }
-
-    /**
-     * Creates the AuthorityProvider instance for the identifier specified.
-     *
-     * @param identifier
-     * @param authorityProviderClassName
-     * @return
-     * @throws Exception
-     */
-    private AuthorityProvider createAuthorityProvider(final String identifier, final String authorityProviderClassName) throws Exception {
-        // get the classloader for the specified authority provider
-        final ClassLoader authorityProviderClassLoader = ExtensionManager.getClassLoader(authorityProviderClassName);
-        if (authorityProviderClassLoader == null) {
-            throw new Exception(String.format("The specified authority provider class '%s' is not known to this nifi.", authorityProviderClassName));
-        }
-
-        // get the current context classloader
-        final ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader();
-
-        final AuthorityProvider instance;
-        try {
-            // set the appropriate class loader
-            Thread.currentThread().setContextClassLoader(authorityProviderClassLoader);
-
-            // attempt to load the class
-            Class<?> rawAuthorityProviderClass = Class.forName(authorityProviderClassName, true, authorityProviderClassLoader);
-            Class<? extends AuthorityProvider> authorityProviderClass = rawAuthorityProviderClass.asSubclass(AuthorityProvider.class);
-
-            // otherwise create a new instance
-            Constructor constructor = authorityProviderClass.getConstructor();
-            instance = (AuthorityProvider) constructor.newInstance();
-
-            // method injection
-            performMethodInjection(instance, authorityProviderClass);
-
-            // field injection
-            performFieldInjection(instance, authorityProviderClass);
-
-            // call post construction lifecycle event
-            instance.initialize(new StandardAuthorityProviderInitializationContext(identifier, this));
-        } finally {
-            if (currentClassLoader != null) {
-                Thread.currentThread().setContextClassLoader(currentClassLoader);
-            }
-        }
-
-        return withNarLoader(instance);
-    }
-
-    /**
-     * Loads the AuthorityProvider configuration.
-     *
-     * @param provider
-     * @return
-     */
-    private AuthorityProviderConfigurationContext loadAuthorityProviderConfiguration(final Provider provider) {
-        final Map<String, String> providerProperties = new HashMap<>();
-
-        for (final Property property : provider.getProperty()) {
-            providerProperties.put(property.getName(), property.getValue());
-        }
-
-        return new StandardAuthorityProviderConfigurationContext(provider.getIdentifier(), providerProperties);
-    }
-
-    /**
-     * Performs method injection.
-     *
-     * @param instance
-     * @param authorityProviderClass
-     * @throws IllegalAccessException
-     * @throws IllegalArgumentException
-     * @throws InvocationTargetException
-     */
-    private void performMethodInjection(final AuthorityProvider instance, final Class authorityProviderClass) throws IllegalAccessException, IllegalArgumentException, InvocationTargetException {
-        for (final Method method : authorityProviderClass.getMethods()) {
-            if (method.isAnnotationPresent(AuthorityProviderContext.class)) {
-                // make the method accessible
-                final boolean isAccessible = method.isAccessible();
-                method.setAccessible(true);
-
-                try {
-                    final Class<?>[] argumentTypes = method.getParameterTypes();
-
-                    // look for setters (single argument)
-                    if (argumentTypes.length == 1) {
-                        final Class<?> argumentType = argumentTypes[0];
-
-                        // look for well known types
-                        if (NiFiProperties.class.isAssignableFrom(argumentType)) {
-                            // nifi properties injection
-                            method.invoke(instance, properties);
-                        } else if (ApplicationContext.class.isAssignableFrom(argumentType)) {
-                            // spring application context injection
-                            method.invoke(instance, applicationContext);
-                        }
-                    }
-                } finally {
-                    method.setAccessible(isAccessible);
-                }
-            }
-        }
-
-        final Class parentClass = authorityProviderClass.getSuperclass();
-        if (parentClass != null && AuthorityProvider.class.isAssignableFrom(parentClass)) {
-            performMethodInjection(instance, parentClass);
-        }
-    }
-
-    /**
-     * Performs field injection.
-     *
-     * @param instance
-     * @param authorityProviderClass
-     * @throws IllegalArgumentException
-     * @throws IllegalAccessException
-     */
-    private void performFieldInjection(final AuthorityProvider instance, final Class authorityProviderClass) throws IllegalArgumentException, IllegalAccessException {
-        for (final Field field : authorityProviderClass.getDeclaredFields()) {
-            if (field.isAnnotationPresent(AuthorityProviderContext.class)) {
-                // make the method accessible
-                final boolean isAccessible = field.isAccessible();
-                field.setAccessible(true);
-
-                try {
-                    // get the type
-                    final Class<?> fieldType = field.getType();
-
-                    // only consider this field if it isn't set yet
-                    if (field.get(instance) == null) {
-                        // look for well known types
-                        if (NiFiProperties.class.isAssignableFrom(fieldType)) {
-                            // nifi properties injection
-                            field.set(instance, properties);
-                        } else if (ApplicationContext.class.isAssignableFrom(fieldType)) {
-                            // spring application context injection
-                            field.set(instance, applicationContext);
-                        }
-                    }
-
-                } finally {
-                    field.setAccessible(isAccessible);
-                }
-            }
-        }
-
-        final Class parentClass = authorityProviderClass.getSuperclass();
-        if (parentClass != null && AuthorityProvider.class.isAssignableFrom(parentClass)) {
-            performFieldInjection(instance, parentClass);
-        }
-    }
-
-    /**
-     * Creates a default provider to use when running unsecurely with no
-     * provider configured.
-     *
-     * @return
-     */
-    private AuthorityProvider createDefaultProvider() {
-        return new AuthorityProvider() {
-            @Override
-            public boolean doesDnExist(String dn) throws AuthorityAccessException {
-                return false;
-            }
-
-            @Override
-            public Set<Authority> getAuthorities(String dn) throws UnknownIdentityException, AuthorityAccessException {
-                return EnumSet.noneOf(Authority.class);
-            }
-
-            @Override
-            public void setAuthorities(String dn, Set<Authority> authorities) throws UnknownIdentityException, AuthorityAccessException {
-            }
-
-            @Override
-            public Set<String> getUsers(Authority authority) throws AuthorityAccessException {
-                return new HashSet<>();
-            }
-
-            @Override
-            public void revokeUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-            }
-
-            @Override
-            public void addUser(String dn, String group) throws IdentityAlreadyExistsException, AuthorityAccessException {
-            }
-
-            @Override
-            public String getGroupForUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-                return null;
-            }
-
-            @Override
-            public void revokeGroup(String group) throws UnknownIdentityException, AuthorityAccessException {
-            }
-
-            @Override
-            public void setUsersGroup(Set<String> dn, String group) throws UnknownIdentityException, AuthorityAccessException {
-            }
-
-            @Override
-            public void ungroupUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-            }
-
-            @Override
-            public void ungroup(String group) throws AuthorityAccessException {
-            }
-
-            @Override
-            public DownloadAuthorization authorizeDownload(List<String> dnChain, Map<String, String> attributes) throws UnknownIdentityException, AuthorityAccessException {
-                return DownloadAuthorization.approved();
-            }
-
-            @Override
-            public void initialize(AuthorityProviderInitializationContext initializationContext) throws ProviderCreationException {
-            }
-
-            @Override
-            public void onConfigured(AuthorityProviderConfigurationContext configurationContext) throws ProviderCreationException {
-            }
-
-            @Override
-            public void preDestruction() throws ProviderDestructionException {
-            }
-        };
-    }
-
-    /**
-     * Decorates the base provider to ensure the nar context classloader is used
-     * when invoking the underlying methods.
-     *
-     * @param baseProvider
-     * @return
-     */
-    public AuthorityProvider withNarLoader(final AuthorityProvider baseProvider) {
-        return new AuthorityProvider() {
-            @Override
-            public boolean doesDnExist(String dn) throws AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    return baseProvider.doesDnExist(dn);
-                }
-            }
-
-            @Override
-            public Set<Authority> getAuthorities(String dn) throws UnknownIdentityException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    return baseProvider.getAuthorities(dn);
-                }
-            }
-
-            @Override
-            public void setAuthorities(String dn, Set<Authority> authorities) throws UnknownIdentityException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.setAuthorities(dn, authorities);
-                }
-            }
-
-            @Override
-            public Set<String> getUsers(Authority authority) throws AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    return baseProvider.getUsers(authority);
-                }
-            }
-
-            @Override
-            public void revokeUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.revokeUser(dn);
-                }
-            }
-
-            @Override
-            public void addUser(String dn, String group) throws IdentityAlreadyExistsException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.addUser(dn, group);
-                }
-            }
-
-            @Override
-            public String getGroupForUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    return baseProvider.getGroupForUser(dn);
-                }
-            }
-
-            @Override
-            public void revokeGroup(String group) throws UnknownIdentityException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.revokeGroup(group);
-                }
-            }
-
-            @Override
-            public void setUsersGroup(Set<String> dns, String group) throws UnknownIdentityException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.setUsersGroup(dns, group);
-                }
-            }
-
-            @Override
-            public void ungroupUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.ungroupUser(dn);
-                }
-            }
-
-            @Override
-            public void ungroup(String group) throws AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.ungroup(group);
-                }
-            }
-
-            @Override
-            public DownloadAuthorization authorizeDownload(List<String> dnChain, Map<String, String> attributes) throws UnknownIdentityException, AuthorityAccessException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    return baseProvider.authorizeDownload(dnChain, attributes);
-                }
-            }
-
-            @Override
-            public void initialize(AuthorityProviderInitializationContext initializationContext) throws ProviderCreationException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.initialize(initializationContext);
-                }
-            }
-
-            @Override
-            public void onConfigured(AuthorityProviderConfigurationContext configurationContext) throws ProviderCreationException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.onConfigured(configurationContext);
-                }
-            }
-
-            @Override
-            public void preDestruction() throws ProviderDestructionException {
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    baseProvider.preDestruction();
-                }
-            }
-        };
-    }
-
-    @Override
-    public Class getObjectType() {
-        return AuthorityProvider.class;
-    }
-
-    @Override
-    public boolean isSingleton() {
-        return true;
-    }
-
-    @Override
-    public void setApplicationContext(ApplicationContext applicationContext) throws BeansException {
-        this.applicationContext = applicationContext;
-    }
-
-    @Override
-    public void destroy() throws Exception {
-        if (authorityProvider != null) {
-            authorityProvider.preDestruction();
-        }
-    }
-
-    public void setProperties(NiFiProperties properties) {
-        this.properties = properties;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderConfigurationContext.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderConfigurationContext.java
deleted file mode 100644
index 0535e27..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderConfigurationContext.java
+++ /dev/null
@@ -1,50 +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.authorization;
-
-import java.util.Collections;
-import java.util.Map;
-
-/**
- *
- */
-public class StandardAuthorityProviderConfigurationContext implements AuthorityProviderConfigurationContext {
-
-    private final String identifier;
-    private final Map<String, String> properties;
-
-    public StandardAuthorityProviderConfigurationContext(String identifier, Map<String, String> properties) {
-        this.identifier = identifier;
-        this.properties = properties;
-    }
-
-    @Override
-    public String getIdentifier() {
-        return identifier;
-    }
-
-    @Override
-    public Map<String, String> getProperties() {
-        return Collections.unmodifiableMap(properties);
-    }
-
-    @Override
-    public String getProperty(String property) {
-        return properties.get(property);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderInitializationContext.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderInitializationContext.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderInitializationContext.java
deleted file mode 100644
index e4b16c4..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderInitializationContext.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.authorization;
-
-/**
- *
- */
-public class StandardAuthorityProviderInitializationContext implements AuthorityProviderInitializationContext {
-
-    private final String identifier;
-    private final AuthorityProviderLookup authorityProviderLookup;
-
-    public StandardAuthorityProviderInitializationContext(String identifier, AuthorityProviderLookup authorityProviderLookup) {
-        this.identifier = identifier;
-        this.authorityProviderLookup = authorityProviderLookup;
-    }
-
-    @Override
-    public String getIdentifier() {
-        return identifier;
-    }
-
-    @Override
-    public AuthorityProviderLookup getAuthorityProviderLookup() {
-        return authorityProviderLookup;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/History.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/History.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/History.java
deleted file mode 100644
index 8536871..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/History.java
+++ /dev/null
@@ -1,56 +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.history;
-
-import java.util.Collection;
-import java.util.Date;
-import org.apache.nifi.action.Action;
-
-/**
- * The result of running an action query.
- */
-public class History {
-
-    private Integer total;
-    private Date lastRefreshed;
-    private Collection<Action> actions;
-
-    public Collection<Action> getActions() {
-        return actions;
-    }
-
-    public void setActions(Collection<Action> actions) {
-        this.actions = actions;
-    }
-
-    public Integer getTotal() {
-        return total;
-    }
-
-    public void setTotal(Integer totalRecordCount) {
-        this.total = totalRecordCount;
-    }
-
-    public Date getLastRefreshed() {
-        return lastRefreshed;
-    }
-
-    public void setLastRefreshed(Date lastRefreshed) {
-        this.lastRefreshed = lastRefreshed;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/HistoryQuery.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/HistoryQuery.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/HistoryQuery.java
deleted file mode 100644
index 53cc13c..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/HistoryQuery.java
+++ /dev/null
@@ -1,99 +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.history;
-
-import java.util.Date;
-
-/**
- *
- */
-public class HistoryQuery {
-
-    private String userName;
-    private String sourceId;
-    private Date startDate;
-    private Date endDate;
-    private Integer offset;
-    private Integer count;
-    private String sortColumn;
-    private String sortOrder;
-
-    public Date getEndDate() {
-        return endDate;
-    }
-
-    public void setEndDate(Date endDate) {
-        this.endDate = endDate;
-    }
-
-    public String getSourceId() {
-        return sourceId;
-    }
-
-    public void setSourceId(String sourceId) {
-        this.sourceId = sourceId;
-    }
-
-    public Date getStartDate() {
-        return startDate;
-    }
-
-    public void setStartDate(Date startDate) {
-        this.startDate = startDate;
-    }
-
-    public String getUserName() {
-        return userName;
-    }
-
-    public void setUserName(String userName) {
-        this.userName = userName;
-    }
-
-    public Integer getOffset() {
-        return offset;
-    }
-
-    public void setOffset(Integer offset) {
-        this.offset = offset;
-    }
-
-    public Integer getCount() {
-        return count;
-    }
-
-    public void setCount(Integer count) {
-        this.count = count;
-    }
-
-    public String getSortColumn() {
-        return sortColumn;
-    }
-
-    public void setSortColumn(String sortColumn) {
-        this.sortColumn = sortColumn;
-    }
-
-    public String getSortOrder() {
-        return sortOrder;
-    }
-
-    public void setSortOrder(String sortOrder) {
-        this.sortOrder = sortOrder;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/PreviousValue.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/PreviousValue.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/PreviousValue.java
deleted file mode 100644
index 6ece5cf..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/PreviousValue.java
+++ /dev/null
@@ -1,54 +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.history;
-
-import java.util.Date;
-
-/**
- *
- */
-public class PreviousValue {
-
-    private String previousValue;
-    private Date timestamp;
-    private String userName;
-
-    public String getPreviousValue() {
-        return previousValue;
-    }
-
-    public void setPreviousValue(String previousValue) {
-        this.previousValue = previousValue;
-    }
-
-    public Date getTimestamp() {
-        return timestamp;
-    }
-
-    public void setTimestamp(Date timestamp) {
-        this.timestamp = timestamp;
-    }
-
-    public String getUserName() {
-        return userName;
-    }
-
-    public void setUserName(String userName) {
-        this.userName = userName;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/AccountStatus.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/AccountStatus.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/AccountStatus.java
deleted file mode 100644
index 195f6bf..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/AccountStatus.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.user;
-
-/**
- * Represents the status of a user's account.
- */
-public enum AccountStatus {
-
-    ACTIVE,
-    PENDING,
-    DISABLED;
-
-    /**
-     * Returns the matching status or null if the specified status does not
-     * match any statuses.
-     *
-     * @param rawStatus
-     * @return
-     */
-    public static AccountStatus valueOfStatus(String rawStatus) {
-        AccountStatus desiredStatus = null;
-
-        for (AccountStatus status : values()) {
-            if (status.toString().equals(rawStatus)) {
-                desiredStatus = status;
-                break;
-            }
-        }
-
-        return desiredStatus;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUser.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUser.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUser.java
deleted file mode 100644
index 415160a..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUser.java
+++ /dev/null
@@ -1,164 +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.user;
-
-import java.io.Serializable;
-import java.util.Date;
-import java.util.EnumSet;
-import java.util.Objects;
-import java.util.Set;
-import org.apache.nifi.authorization.Authority;
-import org.apache.commons.lang3.StringUtils;
-
-/**
- * An NiFiUser.
- */
-public class NiFiUser implements Serializable {
-
-    public static final String ANONYMOUS_USER_DN = "anonymous";
-
-    private String id;
-    private String dn;
-    private String userName;
-    private String userGroup;
-    private String justification;
-
-    private Date creation;
-    private Date lastVerified;
-    private Date lastAccessed;
-
-    private AccountStatus status;
-    private EnumSet<Authority> authorities;
-    
-    private NiFiUser chain;
-
-    /* getters / setters */
-    public Date getCreation() {
-        return creation;
-    }
-
-    public void setCreation(Date creation) {
-        this.creation = creation;
-    }
-
-    public String getDn() {
-        return dn;
-    }
-
-    public void setDn(String dn) {
-        this.dn = dn;
-    }
-
-    public String getUserName() {
-        return userName;
-    }
-
-    public void setUserName(String userName) {
-        this.userName = userName;
-    }
-
-    public String getUserGroup() {
-        return userGroup;
-    }
-
-    public void setUserGroup(String userGroup) {
-        this.userGroup = userGroup;
-    }
-
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    public String getJustification() {
-        return justification;
-    }
-
-    public void setJustification(String justification) {
-        this.justification = justification;
-    }
-
-    public AccountStatus getStatus() {
-        return status;
-    }
-
-    public void setStatus(AccountStatus status) {
-        this.status = status;
-    }
-
-    public Date getLastVerified() {
-        return lastVerified;
-    }
-
-    public void setLastVerified(Date lastVerified) {
-        this.lastVerified = lastVerified;
-    }
-
-    public Date getLastAccessed() {
-        return lastAccessed;
-    }
-
-    public void setLastAccessed(Date lastAccessed) {
-        this.lastAccessed = lastAccessed;
-    }
-
-    public NiFiUser getChain() {
-        return chain;
-    }
-
-    public void setChain(NiFiUser chain) {
-        this.chain = chain;
-    }
-
-    public Set<Authority> getAuthorities() {
-        if (authorities == null) {
-            authorities = EnumSet.noneOf(Authority.class);
-        }
-        return authorities;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (getClass() != obj.getClass()) {
-            return false;
-        }
-        final NiFiUser other = (NiFiUser) obj;
-        if (!Objects.equals(this.dn, other.dn)) {
-            return false;
-        }
-        return true;
-    }
-
-    @Override
-    public int hashCode() {
-        int hash = 7;
-        hash = 53 * hash + Objects.hashCode(this.dn);
-        return hash;
-    }
-
-    @Override
-    public String toString() {
-        return String.format("dn[%s], userName[%s], justification[%s], authorities[%s]", getDn(), getUserName(), getJustification(), StringUtils.join(getAuthorities(), ", "));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUserGroup.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUserGroup.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUserGroup.java
deleted file mode 100644
index 7586fd1..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUserGroup.java
+++ /dev/null
@@ -1,45 +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.user;
-
-import java.util.Set;
-
-/**
- *
- */
-public class NiFiUserGroup {
-
-    private String group;
-    private Set<NiFiUser> users;
-
-    public String getGroup() {
-        return group;
-    }
-
-    public void setGroup(String group) {
-        this.group = group;
-    }
-
-    public Set<NiFiUser> getUsers() {
-        return users;
-    }
-
-    public void setUsers(Set<NiFiUser> users) {
-        this.users = users;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/resources/nifi-administration-context.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/resources/nifi-administration-context.xml b/nar-bundles/framework-bundle/framework/administration/src/main/resources/nifi-administration-context.xml
deleted file mode 100644
index a36619f..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/resources/nifi-administration-context.xml
+++ /dev/null
@@ -1,62 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<beans default-lazy-init="true"
-       xmlns="http://www.springframework.org/schema/beans"
-       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xmlns:context="http://www.springframework.org/schema/context"
-       xmlns:aop="http://www.springframework.org/schema/aop"
-       xsi:schemaLocation="http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-3.1.xsd
-    http://www.springframework.org/schema/context http://www.springframework.org/schema/context/spring-context-3.1.xsd
-    http://www.springframework.org/schema/aop http://www.springframework.org/schema/aop/spring-aop-3.1.xsd">
-
-    <!-- user authority provider -->
-    <bean id="authorityProvider" class="org.apache.nifi.authorization.AuthorityProviderFactoryBean" depends-on="clusterManager flowController">
-        <property name="properties" ref="nifiProperties"/>
-    </bean>
-
-    <!-- initialize the user data source -->
-    <bean id="userDataSource" class="org.apache.nifi.admin.UserDataSourceFactoryBean" destroy-method="shutdown">
-        <property name="properties" ref="nifiProperties"/>
-    </bean>
-
-    <!-- initialize the data source -->
-    <bean id="auditDataSource" class="org.apache.nifi.admin.AuditDataSourceFactoryBean" destroy-method="shutdown" depends-on="userDataSource">
-        <property name="properties" ref="nifiProperties"/>
-    </bean>
-
-    <!-- initialize the user transaction builder -->
-    <bean id="userTransactionBuilder" class="org.apache.nifi.admin.service.transaction.impl.StandardTransactionBuilder">
-        <property name="authorityProvider" ref="authorityProvider"/>
-        <property name="dataSource" ref="userDataSource"/>
-    </bean>
-    
-    <!-- initialize the audit transaction builder -->
-    <bean id="auditTransactionBuilder" class="org.apache.nifi.admin.service.transaction.impl.StandardTransactionBuilder">
-        <property name="authorityProvider" ref="authorityProvider"/>
-        <property name="dataSource" ref="auditDataSource"/>
-    </bean>
-
-    <!-- administration service -->
-    <bean id="userService" class="org.apache.nifi.admin.service.impl.StandardUserService" init-method="seedUserAccounts">
-        <property name="transactionBuilder" ref="userTransactionBuilder"/>
-        <property name="properties" ref="nifiProperties"/>
-    </bean>
-
-    <!-- audit service -->
-    <bean id="auditService" class="org.apache.nifi.admin.service.impl.StandardAuditService">
-        <property name="transactionBuilder" ref="auditTransactionBuilder"/>
-    </bean>
-</beans>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/main/xsd/authority-providers.xsd
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/xsd/authority-providers.xsd b/nar-bundles/framework-bundle/framework/administration/src/main/xsd/authority-providers.xsd
deleted file mode 100644
index 122fa2c..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/main/xsd/authority-providers.xsd
+++ /dev/null
@@ -1,49 +0,0 @@
-<?xml version="1.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.
--->
-<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema">
-    <!-- role -->
-    <xs:complexType name="Provider">
-        <xs:sequence>
-            <xs:element name="identifier" type="NonEmptyStringType"/>
-            <xs:element name="class" type="NonEmptyStringType"/>
-            <xs:element name="property" type="Property" minOccurs="0" maxOccurs="unbounded" />
-        </xs:sequence>
-    </xs:complexType>
-
-    <!-- Name/Value properties-->
-    <xs:complexType name="Property">
-        <xs:simpleContent>
-            <xs:extension base="xs:string">
-                <xs:attribute name="name" type="NonEmptyStringType"></xs:attribute>
-            </xs:extension>
-        </xs:simpleContent>
-    </xs:complexType>
-
-    <xs:simpleType name="NonEmptyStringType">
-        <xs:restriction base="xs:string">
-            <xs:minLength value="1"/>
-        </xs:restriction>
-    </xs:simpleType>
-
-    <!-- users -->
-    <xs:element name="authorityProviders">
-        <xs:complexType>
-            <xs:sequence>
-                <xs:element name="provider" type="Provider" minOccurs="0" maxOccurs="unbounded"/>
-            </xs:sequence>
-        </xs:complexType>
-    </xs:element>
-</xs:schema>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/AuthorizeUserActionTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/AuthorizeUserActionTest.java b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/AuthorizeUserActionTest.java
deleted file mode 100644
index 5797705..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/AuthorizeUserActionTest.java
+++ /dev/null
@@ -1,433 +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.admin.service.action;
-
-import java.util.Date;
-import java.util.EnumSet;
-import java.util.Set;
-import org.apache.nifi.admin.dao.AuthorityDAO;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.admin.service.AccountDisabledException;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.admin.service.AccountPendingException;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.authorization.Authority;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.exception.AuthorityAccessException;
-import org.apache.nifi.authorization.exception.UnknownIdentityException;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUser;
-import org.apache.commons.lang3.StringUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- *
- */
-public class AuthorizeUserActionTest {
-
-    private static final String USER_ID_6 = "6";
-    private static final String USER_ID_7 = "7";
-    private static final String USER_ID_8 = "8";
-    private static final String USER_ID_9 = "9";
-    private static final String USER_ID_10 = "10";
-    private static final String USER_ID_11 = "11";
-
-    private static final String USER_DN_1 = "authority access exception while searching for user";
-    private static final String USER_DN_2 = "unknown user";
-    private static final String USER_DN_3 = "user removed after checking existence";
-    private static final String USER_DN_4 = "access exception getting authorities";
-    private static final String USER_DN_5 = "error creating user account";
-    private static final String USER_DN_6 = "create user general sequence";
-    private static final String USER_DN_7 = "existing user requires verification";
-    private static final String USER_DN_8 = "existing user does not require verification";
-    private static final String USER_DN_9 = "existing pending user";
-    private static final String USER_DN_10 = "existing disabled user";
-    private static final String USER_DN_11 = "existing user is now unknown in the authority provider";
-
-    private DAOFactory daoFactory;
-    private UserDAO userDao;
-    private AuthorityDAO authorityDao;
-    private AuthorityProvider authorityProvider;
-
-    @Before
-    public void setup() throws Exception {
-        // mock the user dao
-        userDao = Mockito.mock(UserDAO.class);
-        Mockito.doAnswer(new Answer<NiFiUser>() {
-            @Override
-            public NiFiUser answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String id = (String) args[0];
-
-                NiFiUser user = null;
-                if (USER_ID_7.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_7);
-                    user.setDn(USER_DN_7);
-                    user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
-                } else if (USER_ID_8.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_8);
-                    user.setDn(USER_DN_8);
-                    user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
-                    user.setLastVerified(new Date());
-                } else if (USER_ID_11.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(USER_ID_11);
-                    user.setDn(USER_DN_11);
-                    user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
-                    user.setStatus(AccountStatus.ACTIVE);
-                }
-
-                return user;
-            }
-        }).when(userDao).findUserById(Mockito.anyString());
-        Mockito.doAnswer(new Answer<NiFiUser>() {
-            @Override
-            public NiFiUser answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String dn = (String) args[0];
-
-                NiFiUser user = null;
-                switch (dn) {
-                    case USER_DN_7:
-                        user = new NiFiUser();
-                        user.setId(USER_ID_7);
-                        user.setDn(USER_DN_7);
-                        user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
-                        break;
-                    case USER_DN_8:
-                        user = new NiFiUser();
-                        user.setId(USER_ID_8);
-                        user.setDn(USER_DN_8);
-                        user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
-                        user.setLastVerified(new Date());
-                        break;
-                    case USER_DN_9:
-                        user = new NiFiUser();
-                        user.setId(USER_ID_9);
-                        user.setDn(USER_DN_9);
-                        user.setStatus(AccountStatus.PENDING);
-                        break;
-                    case USER_DN_10:
-                        user = new NiFiUser();
-                        user.setId(USER_ID_10);
-                        user.setDn(USER_DN_10);
-                        user.setStatus(AccountStatus.DISABLED);
-                        break;
-                    case USER_DN_11:
-                        user = new NiFiUser();
-                        user.setId(USER_ID_11);
-                        user.setDn(USER_DN_11);
-                        user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR));
-                        user.setStatus(AccountStatus.ACTIVE);
-                        break;
-                }
-
-                return user;
-            }
-        }).when(userDao).findUserByDn(Mockito.anyString());
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                NiFiUser user = (NiFiUser) args[0];
-                switch (user.getDn()) {
-                    case USER_DN_5:
-                        throw new DataAccessException();
-                    case USER_DN_6:
-                        user.setId(USER_ID_6);
-                        break;
-                }
-
-                // do nothing
-                return null;
-            }
-        }).when(userDao).createUser(Mockito.any(NiFiUser.class));
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                NiFiUser user = (NiFiUser) args[0];
-
-                // do nothing
-                return null;
-            }
-        }).when(userDao).updateUser(Mockito.any(NiFiUser.class));
-
-        // mock the authority dao
-        authorityDao = Mockito.mock(AuthorityDAO.class);
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                Set<Authority> authorities = (Set<Authority>) args[0];
-                String id = (String) args[1];
-
-                // do nothing
-                return null;
-            }
-        }).when(authorityDao).createAuthorities(Mockito.anySetOf(Authority.class), Mockito.anyString());
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                Set<Authority> authorities = (Set<Authority>) args[0];
-                String id = (String) args[1];
-
-                // do nothing
-                return null;
-            }
-        }).when(authorityDao).deleteAuthorities(Mockito.anySetOf(Authority.class), Mockito.anyString());
-
-        // mock the dao factory
-        daoFactory = Mockito.mock(DAOFactory.class);
-        Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao);
-        Mockito.when(daoFactory.getAuthorityDAO()).thenReturn(authorityDao);
-
-        // mock the authority provider
-        authorityProvider = Mockito.mock(AuthorityProvider.class);
-        Mockito.doAnswer(new Answer<Boolean>() {
-            @Override
-            public Boolean answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String dn = (String) args[0];
-                switch (dn) {
-                    case USER_DN_1:
-                        throw new AuthorityAccessException(StringUtils.EMPTY);
-                    case USER_DN_2:
-                        return false;
-                }
-
-                return true;
-            }
-        }).when(authorityProvider).doesDnExist(Mockito.anyString());
-        Mockito.doAnswer(new Answer<Set<Authority>>() {
-            @Override
-            public Set<Authority> answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String dn = (String) args[0];
-                Set<Authority> authorities = EnumSet.noneOf(Authority.class);
-                switch (dn) {
-                    case USER_DN_3:
-                        throw new UnknownIdentityException(StringUtils.EMPTY);
-                    case USER_DN_4:
-                        throw new AuthorityAccessException(StringUtils.EMPTY);
-                    case USER_DN_6:
-                        authorities.add(Authority.ROLE_MONITOR);
-                        break;
-                    case USER_DN_7:
-                        authorities.add(Authority.ROLE_DFM);
-                        break;
-                    case USER_DN_9:
-                        throw new UnknownIdentityException(StringUtils.EMPTY);
-                    case USER_DN_10:
-                        throw new UnknownIdentityException(StringUtils.EMPTY);
-                    case USER_DN_11:
-                        throw new UnknownIdentityException(StringUtils.EMPTY);
-                }
-
-                return authorities;
-            }
-        }).when(authorityProvider).getAuthorities(Mockito.anyString());
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String dn = (String) args[0];
-                Set<Authority> authorites = (Set<Authority>) args[1];
-
-                // do nothing
-                return null;
-            }
-        }).when(authorityProvider).setAuthorities(Mockito.anyString(), Mockito.anySet());
-    }
-
-    /**
-     * Tests AuthorityAccessException in doesDnExist.
-     *
-     * @throws Exception
-     */
-    @Test(expected = AdministrationException.class)
-    public void testAuthorityAccessExceptionInDoesDnExist() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_1, 0);
-        authorizeUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Test unknown user in the authority provider.
-     *
-     * @throws Exception
-     */
-    @Test(expected = AccountNotFoundException.class)
-    public void testUnknownUser() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_2, 0);
-        authorizeUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Test a user thats been removed after checking their existence.
-     *
-     * @throws Exception
-     */
-    @Test(expected = AccountNotFoundException.class)
-    public void testUserRemovedAfterCheckingExistence() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_3, 0);
-        authorizeUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Testing AuthorityAccessException when getting authorities.
-     *
-     * @throws Exception
-     */
-    @Test(expected = AdministrationException.class)
-    public void testAuthorityAccessException() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_4, 0);
-        authorizeUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Testing DataAccessException while creating user accounts.
-     *
-     * @throws Exception
-     */
-    @Test(expected = DataAccessException.class)
-    public void testErrorCreatingUserAccount() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_5, 0);
-        authorizeUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Tests the general case when a user account is created.
-     *
-     * @throws Exception
-     */
-    @Test
-    public void testAccountCreation() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_6, 0);
-        NiFiUser user = authorizeUser.execute(daoFactory, authorityProvider);
-
-        // verify the user
-        Assert.assertEquals(USER_DN_6, user.getDn());
-        Assert.assertEquals(1, user.getAuthorities().size());
-        Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_MONITOR));
-
-        // verify interaction with dao and provider
-        Mockito.verify(userDao, Mockito.times(1)).createUser(user);
-    }
-
-    /**
-     * Tests the general case when there is an existing user account that
-     * requires verification.
-     *
-     * @throws Exception
-     */
-    @Test
-    public void testExistingUserRequiresVerification() throws Exception {
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_7, 0);
-        NiFiUser user = authorizeUser.execute(daoFactory, authorityProvider);
-
-        // verify the user
-        Assert.assertEquals(USER_DN_7, user.getDn());
-        Assert.assertEquals(1, user.getAuthorities().size());
-        Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_DFM));
-
-        // verify interaction with dao and provider
-        Mockito.verify(userDao, Mockito.times(1)).updateUser(user);
-        Mockito.verify(authorityDao, Mockito.times(1)).createAuthorities(EnumSet.of(Authority.ROLE_DFM), USER_ID_7);
-    }
-
-    /**
-     * Tests the general case when there is an existing user account that does
-     * not require verification.
-     *
-     * @throws Exception
-     */
-    @Test
-    public void testExistingUserNoVerification() throws Exception {
-        // disabling verification by passing in a large cache duration
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_8, Integer.MAX_VALUE);
-        NiFiUser user = authorizeUser.execute(daoFactory, authorityProvider);
-
-        // verify the user
-        Assert.assertEquals(USER_DN_8, user.getDn());
-        Assert.assertEquals(1, user.getAuthorities().size());
-        Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_MONITOR));
-
-        // verify interaction with dao and provider
-        Mockito.verify(userDao, Mockito.times(1)).updateUser(user);
-        Mockito.verify(authorityDao, Mockito.never()).createAuthorities(Mockito.anySet(), Mockito.eq(USER_ID_8));
-        Mockito.verify(authorityDao, Mockito.never()).deleteAuthorities(Mockito.anySet(), Mockito.eq(USER_ID_8));
-    }
-
-    /**
-     * Tests existing users whose accounts are in a pending status.
-     *
-     * @throws Exception
-     */
-    @Test(expected = AccountPendingException.class)
-    public void testExistingPendingUser() throws Exception {
-        // disabling verification by passing in a large cache duration
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_9, Integer.MAX_VALUE);
-        authorizeUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Tests existing users whose accounts are in a disabled status.
-     *
-     * @throws Exception
-     */
-    @Test(expected = AccountDisabledException.class)
-    public void testExistingDisabledUser() throws Exception {
-        // disabling verification by passing in a large cache duration
-        AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_10, Integer.MAX_VALUE);
-        authorizeUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Tests the general case where there is an active user that has been
-     * removed from the authority provider.
-     *
-     * @throws Exception
-     */
-    @Test
-    public void testExistingActiveUserNotFoundInProvider() throws Exception {
-        try {
-            AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_11, 0);
-            authorizeUser.execute(daoFactory, authorityProvider);
-
-            Assert.fail();
-        } catch (AccountDisabledException ade) {
-            ArgumentCaptor<NiFiUser> user = ArgumentCaptor.forClass(NiFiUser.class);
-
-            // verify interaction with dao
-            Mockito.verify(userDao, Mockito.times(1)).updateUser(user.capture());
-
-            // verify user
-            Assert.assertEquals(AccountStatus.DISABLED, user.getValue().getStatus());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/CreateUserActionTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/CreateUserActionTest.java b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/CreateUserActionTest.java
deleted file mode 100644
index 3d2081b..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/CreateUserActionTest.java
+++ /dev/null
@@ -1,144 +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.admin.service.action;
-
-import java.util.EnumSet;
-import java.util.Set;
-import org.apache.nifi.admin.dao.AuthorityDAO;
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.authorization.Authority;
-import org.apache.nifi.user.NiFiUser;
-import org.apache.commons.lang3.StringUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Test cases for creating a user.
- */
-public class CreateUserActionTest {
-
-    private String USER_ID_2 = "2";
-    private String USER_ID_3 = "3";
-
-    private String USER_DN_1 = "data access exception when creating user";
-    private String USER_DN_3 = "general create user case";
-
-    private DAOFactory daoFactory;
-    private UserDAO userDao;
-    private AuthorityDAO authorityDao;
-
-    @Before
-    public void setup() throws Exception {
-        // mock the user dao
-        userDao = Mockito.mock(UserDAO.class);
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                NiFiUser user = (NiFiUser) args[0];
-
-                if (USER_DN_1.equals(user.getDn())) {
-                    throw new DataAccessException();
-                } else if (USER_DN_3.equals(user.getDn())) {
-                    user.setId(USER_ID_3);
-                }
-
-                // do nothing
-                return null;
-            }
-        }).when(userDao).createUser(Mockito.any(NiFiUser.class));
-
-        // mock the authority dao
-        authorityDao = Mockito.mock(AuthorityDAO.class);
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                Set<Authority> authorities = (Set<Authority>) args[0];
-                String id = (String) args[1];
-
-                if (USER_ID_2.equals(id)) {
-                    throw new DataAccessException(StringUtils.EMPTY);
-                }
-
-                // do nothing
-                return null;
-            }
-        }).when(authorityDao).createAuthorities(Mockito.anySetOf(Authority.class), Mockito.anyString());
-
-        // mock the dao factory
-        daoFactory = Mockito.mock(DAOFactory.class);
-        Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao);
-        Mockito.when(daoFactory.getAuthorityDAO()).thenReturn(authorityDao);
-    }
-
-    /**
-     * Tests DataAccessExceptions that occur while creating user accounts.
-     *
-     * @throws Exception
-     */
-    @Test(expected = DataAccessException.class)
-    public void testExceptionCreatingUser() throws Exception {
-        NiFiUser user = new NiFiUser();
-        user.setDn(USER_DN_1);
-
-        CreateUserAction createUser = new CreateUserAction(user);
-        createUser.execute(daoFactory, null);
-    }
-
-    /**
-     * Tests DataAccessExceptions that occur while create user authorities.
-     *
-     * @throws Exception
-     */
-    @Test(expected = DataAccessException.class)
-    public void testExceptionCreatingAuthoroties() throws Exception {
-        NiFiUser user = new NiFiUser();
-        user.setId(USER_ID_2);
-
-        CreateUserAction createUser = new CreateUserAction(user);
-        createUser.execute(daoFactory, null);
-    }
-
-    /**
-     * General case for creating a user.
-     *
-     * @throws Exception
-     */
-    @Test
-    public void testCreateUserAccount() throws Exception {
-        NiFiUser user = new NiFiUser();
-        user.setDn(USER_DN_3);
-        user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_DFM, Authority.ROLE_ADMIN));
-
-        CreateUserAction createUser = new CreateUserAction(user);
-        createUser.execute(daoFactory, null);
-
-        // verify the user
-        Assert.assertEquals(USER_ID_3, user.getId());
-
-        // verify interaction with dao
-        Mockito.verify(userDao, Mockito.times(1)).createUser(user);
-        Mockito.verify(authorityDao, Mockito.times(1)).createAuthorities(user.getAuthorities(), USER_ID_3);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/300952a9/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/DisableUserActionTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/DisableUserActionTest.java b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/DisableUserActionTest.java
deleted file mode 100644
index de85298..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/DisableUserActionTest.java
+++ /dev/null
@@ -1,171 +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.admin.service.action;
-
-import org.apache.nifi.admin.dao.DAOFactory;
-import org.apache.nifi.admin.dao.DataAccessException;
-import org.apache.nifi.admin.dao.UserDAO;
-import org.apache.nifi.admin.service.AccountNotFoundException;
-import org.apache.nifi.admin.service.AdministrationException;
-import org.apache.nifi.authorization.AuthorityProvider;
-import org.apache.nifi.authorization.exception.AuthorityAccessException;
-import org.apache.nifi.user.AccountStatus;
-import org.apache.nifi.user.NiFiUser;
-import org.apache.commons.lang3.StringUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- *
- */
-public class DisableUserActionTest {
-
-    private static final String USER_ID_1 = "1";
-    private static final String USER_ID_2 = "2";
-    private static final String USER_ID_3 = "3";
-    private static final String USER_ID_4 = "4";
-
-    private static final String USER_DN_3 = "authority access exception";
-    private static final String USER_DN_4 = "general disable user case";
-
-    private DAOFactory daoFactory;
-    private UserDAO userDao;
-    private AuthorityProvider authorityProvider;
-
-    @Before
-    public void setup() throws Exception {
-        // mock the user dao
-        userDao = Mockito.mock(UserDAO.class);
-        Mockito.doAnswer(new Answer<NiFiUser>() {
-            @Override
-            public NiFiUser answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String id = (String) args[0];
-
-                NiFiUser user = null;
-                if (USER_ID_1.equals(id)) {
-                    // leave user uninitialized
-                } else if (USER_ID_2.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(id);
-                } else if (USER_ID_3.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(id);
-                    user.setDn(USER_DN_3);
-                } else if (USER_ID_4.equals(id)) {
-                    user = new NiFiUser();
-                    user.setId(id);
-                    user.setDn(USER_DN_4);
-                    user.setStatus(AccountStatus.ACTIVE);
-                }
-                return user;
-            }
-        }).when(userDao).findUserById(Mockito.anyString());
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                NiFiUser user = (NiFiUser) args[0];
-
-                if (USER_ID_2.equals(user.getId())) {
-                    throw new DataAccessException(StringUtils.EMPTY);
-                }
-
-                // do nothing
-                return null;
-            }
-        }).when(userDao).updateUser(Mockito.any(NiFiUser.class));
-
-        // mock the dao factory
-        daoFactory = Mockito.mock(DAOFactory.class);
-        Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao);
-
-        // mock the authority provider
-        authorityProvider = Mockito.mock(AuthorityProvider.class);
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object[] args = invocation.getArguments();
-                String dn = (String) args[0];
-
-                if (USER_DN_3.equals(dn)) {
-                    throw new AuthorityAccessException(StringUtils.EMPTY);
-                }
-
-                // do nothing
-                return null;
-            }
-        }).when(authorityProvider).revokeUser(Mockito.anyString());
-    }
-
-    /**
-     * Tests the case when the user account is unknown.
-     *
-     * @throws Exception
-     */
-    @Test(expected = AccountNotFoundException.class)
-    public void testUnknownUserAccount() throws Exception {
-        DisableUserAction disableUser = new DisableUserAction(USER_ID_1);
-        disableUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Tests the case when a DataAccessException is thrown by the userDao.
-     *
-     * @throws Exception
-     */
-    @Test(expected = DataAccessException.class)
-    public void testDataAccessExceptionInUserDao() throws Exception {
-        DisableUserAction disableUser = new DisableUserAction(USER_ID_2);
-        disableUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Tests the case when a AuthorityAccessException is thrown by the provider.
-     *
-     * @throws Exception
-     */
-    @Test(expected = AdministrationException.class)
-    public void testAuthorityAccessExceptionInProvider() throws Exception {
-        DisableUserAction disableUser = new DisableUserAction(USER_ID_3);
-        disableUser.execute(daoFactory, authorityProvider);
-    }
-
-    /**
-     * Tests the general case when the user is disabled.
-     *
-     * @throws Exception
-     */
-    @Test
-    public void testDisableUser() throws Exception {
-        DisableUserAction disableUser = new DisableUserAction(USER_ID_4);
-        NiFiUser user = disableUser.execute(daoFactory, authorityProvider);
-
-        // verify the user
-        Assert.assertEquals(USER_ID_4, user.getId());
-        Assert.assertEquals(USER_DN_4, user.getDn());
-        Assert.assertEquals(AccountStatus.DISABLED, user.getStatus());
-
-        // verify the interaction with the dao and provider
-        Mockito.verify(userDao, Mockito.times(1)).updateUser(user);
-        Mockito.verify(authorityProvider, Mockito.times(1)).revokeUser(USER_DN_4);
-    }
-}