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 2014/12/08 21:29:24 UTC

[01/51] [partial] incubator-nifi git commit: Initial code contribution

Repository: incubator-nifi
Updated Branches:
  refs/heads/master [created] 4d998c12c


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..17a1702
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
@@ -0,0 +1,1026 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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());
+            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));
+    }
+}


[29/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..4b89655
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java
@@ -0,0 +1,1056 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..4e2cc26
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardAuthorityDAO.java
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ea7c1a1
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java
@@ -0,0 +1,634 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..e8b3d10
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountDisabledException.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..88287ce
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountNotFoundException.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..dacc483
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountPendingException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..c0e8ac1
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AdministrationException.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..0843bd8
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AuditService.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+}


[34/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedSetCacheClientService.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedSetCacheClientService.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedSetCacheClientService.java
new file mode 100644
index 0000000..1d7c94c
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedSetCacheClientService.java
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.client;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.annotation.OnConfigured;
+import org.apache.nifi.distributed.cache.protocol.ProtocolHandshake;
+import org.apache.nifi.distributed.cache.protocol.exception.HandshakeException;
+import org.apache.nifi.io.ByteArrayOutputStream;
+import org.apache.nifi.io.DataOutputStream;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.remote.StandardVersionNegotiator;
+import org.apache.nifi.remote.VersionNegotiator;
+import org.apache.nifi.ssl.SSLContextService;
+import org.apache.nifi.ssl.SSLContextService.ClientAuth;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DistributedSetCacheClientService extends AbstractControllerService implements DistributedSetCacheClient {
+
+    private static final Logger logger = LoggerFactory.getLogger(DistributedMapCacheClientService.class);
+
+    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor.Builder()
+            .name("Server Hostname")
+            .description("The name of the server that is running the DistributedSetCacheServer service")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+            .name("Server Port")
+            .description("The port on the remote server that is to be used when communicating with the DistributedSetCacheServer service")
+            .required(true)
+            .addValidator(StandardValidators.PORT_VALIDATOR)
+            .defaultValue("4557")
+            .build();
+    public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
+            .name("SSL Context Service")
+            .description(
+                    "If specified, indicates the SSL Context Service that is used to communicate with the remote server. If not specified, communications will not be encrypted")
+            .required(false)
+            .addValidator(StandardValidators.createControllerServiceExistsValidator(SSLContextService.class))
+            .defaultValue(null)
+            .build();
+    public static final PropertyDescriptor COMMUNICATIONS_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("Communications Timeout")
+            .description(
+                    "Specifices how long to wait when communicating with the remote server before determining that there is a communications failure if data cannot be sent or received")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .build();
+
+    private final BlockingQueue<CommsSession> queue = new LinkedBlockingQueue<>();
+    private volatile ConfigurationContext configContext;
+    private volatile boolean closed = false;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(HOSTNAME);
+        descriptors.add(PORT);
+        descriptors.add(SSL_CONTEXT_SERVICE);
+        descriptors.add(COMMUNICATIONS_TIMEOUT);
+        return descriptors;
+    }
+
+    @OnConfigured
+    public void onConfigured(final ConfigurationContext context) {
+        this.configContext = context;
+    }
+
+    public CommsSession createCommsSession(final ConfigurationContext context) throws IOException {
+        final String hostname = context.getProperty(HOSTNAME).getValue();
+        final int port = context.getProperty(PORT).asInteger();
+        final long timeoutMillis = context.getProperty(COMMUNICATIONS_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS);
+        final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+
+        final CommsSession commsSession;
+        if (sslContextService == null) {
+            commsSession = new StandardCommsSession(hostname, port);
+        } else {
+            commsSession = new SSLCommsSession(sslContextService.createSSLContext(ClientAuth.REQUIRED), hostname, port);
+        }
+
+        commsSession.setTimeout(timeoutMillis, TimeUnit.MILLISECONDS);
+        return commsSession;
+    }
+
+    private CommsSession leaseCommsSession() throws IOException {
+        CommsSession session = queue.poll();
+        if (session != null && !session.isClosed()) {
+            return session;
+        }
+
+        session = createCommsSession(configContext);
+        final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(1);
+        try {
+            ProtocolHandshake.initiateHandshake(session.getInputStream(), session.getOutputStream(), versionNegotiator);
+        } catch (final HandshakeException e) {
+            try {
+                session.close();
+            } catch (final IOException ioe) {
+            }
+
+            throw new IOException(e);
+        }
+
+        return session;
+    }
+
+    @Override
+    public <T> boolean addIfAbsent(final T value, final Serializer<T> serializer) throws IOException {
+        return invokeRemoteBoolean("addIfAbsent", value, serializer);
+    }
+
+    @Override
+    public <T> boolean contains(final T value, final Serializer<T> serializer) throws IOException {
+        return invokeRemoteBoolean("contains", value, serializer);
+    }
+
+    @Override
+    public <T> boolean remove(final T value, final Serializer<T> serializer) throws IOException {
+        return invokeRemoteBoolean("remove", value, serializer);
+    }
+
+    @Override
+    public void close() throws IOException {
+        this.closed = true;
+
+        CommsSession commsSession;
+        while ((commsSession = queue.poll()) != null) {
+            try (final DataOutputStream dos = new DataOutputStream(commsSession.getOutputStream())) {
+                dos.writeUTF("close");
+                dos.flush();
+                commsSession.close();
+            } catch (final IOException e) {
+            }
+        }
+        logger.info("Closed {}", new Object[] { getIdentifier() });
+    }
+
+    @Override
+    protected void finalize() throws Throwable {
+        if (!closed)
+            close();
+        logger.debug("Finalize called");
+    }
+
+    private <T> boolean invokeRemoteBoolean(final String methodName, final T value, final Serializer<T> serializer) throws IOException {
+        if (closed) {
+            throw new IllegalStateException("Client is closed");
+        }
+
+        final CommsSession session = leaseCommsSession();
+        try {
+            final DataOutputStream dos = new DataOutputStream(session.getOutputStream());
+            dos.writeUTF(methodName);
+
+            final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+            serializer.serialize(value, baos);
+            dos.writeInt(baos.size());
+            baos.writeTo(dos);
+            dos.flush();
+
+            final DataInputStream dis = new DataInputStream(session.getInputStream());
+            return dis.readBoolean();
+        } catch (final IOException ioe) {
+            try {
+                session.close();
+            } catch (final IOException ignored) {
+            }
+
+            throw ioe;
+        } finally {
+            if (!session.isClosed()) {
+                if (this.closed) {
+                    try {
+                        session.close();
+                    } catch (final IOException ioe) {
+                    }
+                } else {
+                    queue.offer(session);
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/SSLCommsSession.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/SSLCommsSession.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/SSLCommsSession.java
new file mode 100644
index 0000000..c8be082
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/SSLCommsSession.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.client;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.TimeUnit;
+
+import javax.net.ssl.SSLContext;
+
+import org.apache.nifi.io.BufferedInputStream;
+import org.apache.nifi.io.BufferedOutputStream;
+import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel;
+import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelInputStream;
+import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelOutputStream;
+
+public class SSLCommsSession implements CommsSession {
+    private final SSLSocketChannel sslSocketChannel;
+    private final SSLContext sslContext;
+    private final String hostname;
+    private final int port;
+    
+    private final SSLSocketChannelInputStream in;
+    private final BufferedInputStream bufferedIn;
+    
+    private final SSLSocketChannelOutputStream out;
+    private final BufferedOutputStream bufferedOut;
+
+    public SSLCommsSession(final SSLContext sslContext, final String hostname, final int port) throws IOException { 
+        sslSocketChannel = new SSLSocketChannel(sslContext, hostname, port, true);
+        
+        in = new SSLSocketChannelInputStream(sslSocketChannel);
+        bufferedIn = new BufferedInputStream(in);
+        
+        out = new SSLSocketChannelOutputStream(sslSocketChannel);
+        bufferedOut = new BufferedOutputStream(out);
+        
+        this.sslContext = sslContext;
+        this.hostname = hostname;
+        this.port = port;
+    }
+    
+    @Override
+    public void interrupt() {
+        sslSocketChannel.interrupt();
+    }
+    
+    @Override
+    public void close() throws IOException {
+        sslSocketChannel.close();
+    }
+
+    @Override
+    public void setTimeout(final long value, final TimeUnit timeUnit) {
+        sslSocketChannel.setTimeout((int) TimeUnit.MILLISECONDS.convert(value, timeUnit));
+    }
+
+    @Override
+    public InputStream getInputStream() throws IOException {
+        return bufferedIn;
+    }
+
+    @Override
+    public OutputStream getOutputStream() throws IOException {
+        return bufferedOut;
+    }
+
+    @Override
+    public boolean isClosed() {
+        return sslSocketChannel.isClosed();
+    }
+    
+    @Override
+    public String getHostname() {
+        return hostname;
+    }
+    
+    @Override
+    public int getPort() {
+        return port;
+    }
+    @Override
+    public SSLContext getSSLContext() {
+        return sslContext;
+    }
+    @Override
+    public long getTimeout(final TimeUnit timeUnit) {
+        return timeUnit.convert(sslSocketChannel.getTimeout(), TimeUnit.MILLISECONDS);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/StandardCommsSession.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/StandardCommsSession.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/StandardCommsSession.java
new file mode 100644
index 0000000..bbe2917
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/StandardCommsSession.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.client;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.nio.channels.SocketChannel;
+import java.util.concurrent.TimeUnit;
+
+import javax.net.ssl.SSLContext;
+
+import org.apache.nifi.io.BufferedInputStream;
+import org.apache.nifi.io.BufferedOutputStream;
+import org.apache.nifi.remote.io.InterruptableInputStream;
+import org.apache.nifi.remote.io.InterruptableOutputStream;
+import org.apache.nifi.remote.io.socket.SocketChannelInputStream;
+import org.apache.nifi.remote.io.socket.SocketChannelOutputStream;
+
+public class StandardCommsSession implements CommsSession {
+    private final SocketChannel socketChannel;
+    private final String hostname;
+    private final int port;
+    private volatile long timeoutMillis;
+
+    private final SocketChannelInputStream in;
+    private final InterruptableInputStream bufferedIn;
+
+    private final SocketChannelOutputStream out;
+    private final InterruptableOutputStream bufferedOut;
+
+    public StandardCommsSession(final String hostname, final int port) throws IOException {
+        socketChannel = SocketChannel.open(new InetSocketAddress(hostname, port));
+        socketChannel.configureBlocking(false);
+        in = new SocketChannelInputStream(socketChannel);
+        bufferedIn = new InterruptableInputStream(new BufferedInputStream(in));
+
+        out = new SocketChannelOutputStream(socketChannel);
+        bufferedOut = new InterruptableOutputStream(new BufferedOutputStream(out));
+
+        this.hostname = hostname;
+        this.port = port;
+    }
+
+    @Override
+    public void interrupt() {
+        bufferedIn.interrupt();
+        bufferedOut.interrupt();
+    }
+
+    @Override
+    public void close() throws IOException {
+        socketChannel.close();
+    }
+
+    @Override
+    public void setTimeout(final long value, final TimeUnit timeUnit) {
+        in.setTimeout((int) TimeUnit.MILLISECONDS.convert(value, timeUnit));
+        out.setTimeout((int) TimeUnit.MILLISECONDS.convert(value, timeUnit));
+        timeoutMillis = TimeUnit.MILLISECONDS.convert(value, timeUnit);
+    }
+
+    @Override
+    public InputStream getInputStream() throws IOException {
+        return bufferedIn;
+    }
+
+    @Override
+    public OutputStream getOutputStream() throws IOException {
+        return bufferedOut;
+    }
+
+    @Override
+    public boolean isClosed() {
+        boolean closed = !socketChannel.isConnected();
+        if (!closed) {
+            try {
+                this.in.isDataAvailable();
+            } catch (IOException e) {
+                try {
+                    close();
+                } catch (IOException e1) {
+                }
+                closed = true;
+            }
+        }
+        return closed;
+    }
+
+    @Override
+    public String getHostname() {
+        return hostname;
+    }
+
+    @Override
+    public int getPort() {
+        return port;
+    }
+
+    @Override
+    public SSLContext getSSLContext() {
+        return null;
+    }
+
+    @Override
+    public long getTimeout(final TimeUnit timeUnit) {
+        return timeUnit.convert(timeoutMillis, TimeUnit.MILLISECONDS);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
new file mode 100644
index 0000000..a91f7ee
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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.distributed.cache.client.DistributedSetCacheClientService
+org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/index.html
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/index.html b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/index.html
new file mode 100644
index 0000000..d5f3595
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/index.html
@@ -0,0 +1,78 @@
+<!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>Distributed Map Cache Client Service</title>
+<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
+</head>
+
+<body>
+	<h2>Description:</h2>
+
+	<p>A Controller Service that can be used to communicate with a
+		Distributed Map Cache Server.</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.</p>
+
+	<ul>
+		<li><strong>Server Hostname</strong>
+			<ul>
+				<li>The name of the server that is running the DistributedMapCacheServer service</li>
+				<li>Default value: no default</li>
+				<li>Supports expression language: false</li>
+			</ul></li>
+		<li><strong>Server Port</strong>
+			<ul>
+				<li>The port on the remote server that is to be used when communicating with the 
+				<a href="../nifi.distributed.cache.server.map.DistributedMapCacheServer/index.html">DistributedMapCacheServer</a> service</li>
+				
+				<li>Default value: 4557</li>
+				<li>Supports expression language: false</li>
+			</ul></li>
+		<li>SSL Context Service
+			<ul>
+				<li>If specified, indicates the SSL Context Service that is used to communicate with the remote server. If not specified, communications will not be encrypted
+				<li>Default value: no default</li>
+				<li>Supports expression language: false</li>
+			</ul></li>
+		<li><strong>Communications Timeout</strong>
+			<ul>
+				<li>Specifices how long to wait when communicating with the remote server before determining that there is a communications failure if data cannot be sent or received
+				<li>Default value: 30 secs</li>
+				<li>Supports expression language: false</li>
+			</ul></li>
+			
+	</ul>
+
+
+	<i>See Also:</i>
+	<ul>
+		<li><a href="../org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/index.html">Distributed Map Cache Server</a></li>
+		<li><a href="../org.apache.nifi.ssl.StandardSSLContextService/index.html">Standard SSL Context Service</a></li>
+	</ul>
+
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedSetCacheClientService/index.html
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedSetCacheClientService/index.html b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedSetCacheClientService/index.html
new file mode 100755
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/pom.xml b/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/pom.xml
new file mode 100644
index 0000000..bc612ae
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/pom.xml
@@ -0,0 +1,39 @@
+<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>distributed-cache-services-bundle</artifactId>
+		<version>0.0.1-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>distributed-cache-protocol</artifactId>
+	<name>Distributed Cache Protocol</name>
+
+	<description>
+	  	Defines the communications protocol that is used between clients and servers 
+	  	for the Distributed Cache services
+	</description>
+
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>remote-communications-utils</artifactId>
+		</dependency>
+	</dependencies>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/ProtocolHandshake.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/ProtocolHandshake.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/ProtocolHandshake.java
new file mode 100644
index 0000000..da2acad
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/ProtocolHandshake.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.protocol;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.nifi.distributed.cache.protocol.exception.HandshakeException;
+import org.apache.nifi.remote.VersionNegotiator;
+
+public class ProtocolHandshake {
+
+    public static final byte[] MAGIC_HEADER = new byte[] { 'N', 'i', 'F', 'i' };
+    
+    public static final int RESOURCE_OK = 20;
+    public static final int DIFFERENT_RESOURCE_VERSION = 21;
+    public static final int ABORT = 255;
+
+    
+    public static void initiateHandshake(final InputStream in, final OutputStream out, final VersionNegotiator versionNegotiator) throws IOException, HandshakeException {
+        final DataInputStream dis = new DataInputStream(in);
+        final DataOutputStream dos = new DataOutputStream(out);
+        
+        try {
+            dos.write(MAGIC_HEADER);
+            
+            initiateVersionNegotiation(versionNegotiator, dis, dos);
+        } finally {
+            dos.flush();
+        }
+    }
+
+    
+    public static void receiveHandshake(final InputStream in, final OutputStream out, final VersionNegotiator versionNegotiator) throws IOException, HandshakeException {
+        final DataInputStream dis = new DataInputStream(in);
+        final DataOutputStream dos = new DataOutputStream(out);
+        
+        try {
+            final byte[] magicHeaderBuffer = new byte[4];
+            dis.readFully(magicHeaderBuffer);
+            
+            receiveVersionNegotiation(versionNegotiator, dis, dos);
+        } finally {
+            dos.flush();
+        }
+    }
+    
+    
+    private static void initiateVersionNegotiation(final VersionNegotiator negotiator, final DataInputStream dis, final DataOutputStream dos) throws IOException, HandshakeException {
+        // Write the classname of the RemoteStreamCodec, followed by its version
+        dos.writeInt(negotiator.getVersion());
+        dos.flush();
+        
+        // wait for response from server.
+        final int statusCode = dis.read();
+        switch (statusCode) {
+            case RESOURCE_OK:   // server accepted our proposal of codec name/version
+                return;
+            case DIFFERENT_RESOURCE_VERSION:    // server accepted our proposal of codec name but not the version
+                // Get server's preferred version
+                final int newVersion = dis.readInt();
+                
+                // Determine our new preferred version that is no greater than the server's preferred version.
+                final Integer newPreference = negotiator.getPreferredVersion(newVersion);
+                // If we could not agree with server on a version, fail now.
+                if ( newPreference == null ) {
+                    throw new HandshakeException("Could not agree on protocol version");
+                }
+                
+                negotiator.setVersion(newPreference);
+                
+                // Attempt negotiation of resource based on our new preferred version.
+                initiateVersionNegotiation(negotiator, dis, dos);
+            case ABORT:
+                throw new HandshakeException("Remote destination aborted connection with message: " + dis.readUTF());
+            default:
+                throw new HandshakeException("Received unexpected response code " + statusCode + " when negotiating version with remote server");
+        }
+    }
+    
+    private static void receiveVersionNegotiation(final VersionNegotiator negotiator, final DataInputStream dis, final DataOutputStream dos) throws IOException, HandshakeException {
+        final int version = dis.readInt();
+        if ( negotiator.isVersionSupported(version) ) {
+            dos.write(RESOURCE_OK);
+            dos.flush();
+            
+            negotiator.setVersion(version);
+        } else {
+            final Integer preferred = negotiator.getPreferredVersion(version);
+            if ( preferred == null ) {
+                dos.write(ABORT);
+                dos.flush();
+                throw new HandshakeException("Unable to negotiate an acceptable version of the Distributed Cache Protocol");
+            }
+            dos.write(DIFFERENT_RESOURCE_VERSION);
+            dos.writeInt(preferred);
+            dos.flush();
+            
+            receiveVersionNegotiation(negotiator, dis, dos);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/exception/HandshakeException.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/exception/HandshakeException.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/exception/HandshakeException.java
new file mode 100644
index 0000000..8049d42
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/exception/HandshakeException.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.protocol.exception;
+
+public class HandshakeException extends Exception {
+    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/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/pom.xml b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/pom.xml
new file mode 100644
index 0000000..5dec322
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/pom.xml
@@ -0,0 +1,81 @@
+<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>distributed-cache-services-bundle</artifactId>
+		<version>0.0.1-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>distributed-cache-server</artifactId>
+
+	<name>Distributed Cache Server</name>
+	<description>Provides a Controller Service for hosting Distributed Caches</description>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>nifi-api</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>distributed-cache-protocol</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>remote-communications-utils</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>nifi-processor-utils</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>nifi-stream-utils</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>ssl-context-service-api</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>wali</groupId>
+			<artifactId>wali</artifactId>
+		</dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>distributed-cache-client-service-api</artifactId>
+            <scope>test</scope>
+        </dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>distributed-cache-client-service</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>nifi-mock</artifactId>
+		</dependency>
+
+		<dependency>
+		    <groupId>org.apache.nifi</groupId>
+		    <artifactId>ssl-context-service</artifactId>
+		</dependency>
+	</dependencies>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/AbstractCacheServer.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/AbstractCacheServer.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/AbstractCacheServer.java
new file mode 100644
index 0000000..9b4e70e
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/AbstractCacheServer.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.SocketTimeoutException;
+import java.nio.channels.ServerSocketChannel;
+import java.nio.channels.SocketChannel;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArraySet;
+
+import javax.net.ssl.SSLContext;
+
+import org.apache.nifi.distributed.cache.protocol.ProtocolHandshake;
+import org.apache.nifi.distributed.cache.protocol.exception.HandshakeException;
+import org.apache.nifi.io.BufferedInputStream;
+import org.apache.nifi.io.BufferedOutputStream;
+import org.apache.nifi.remote.StandardVersionNegotiator;
+import org.apache.nifi.remote.VersionNegotiator;
+import org.apache.nifi.remote.io.socket.SocketChannelInputStream;
+import org.apache.nifi.remote.io.socket.SocketChannelOutputStream;
+import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel;
+import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelInputStream;
+import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelOutputStream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractCacheServer implements CacheServer {
+
+    private static final Logger logger = LoggerFactory.getLogger(AbstractCacheServer.class);
+
+    private final String identifier;
+    private final int port;
+    private final SSLContext sslContext;
+    protected volatile boolean stopped = false;
+    private final Set<Thread> processInputThreads = new CopyOnWriteArraySet<>();;
+
+    private volatile ServerSocketChannel serverSocketChannel;
+
+    public AbstractCacheServer(final String identifier, final SSLContext sslContext, final int port) {
+        this.identifier = identifier;
+        this.port = port;
+        this.sslContext = sslContext;
+    }
+
+    @Override
+    public void start() throws IOException {
+        serverSocketChannel = ServerSocketChannel.open();
+        serverSocketChannel.configureBlocking(true);
+        serverSocketChannel.bind(new InetSocketAddress(port));
+
+        final Runnable runnable = new Runnable() {
+
+            @Override
+            public void run() {
+                while (true) {
+                    final SocketChannel socketChannel;
+                    try {
+                        socketChannel = serverSocketChannel.accept();
+                        logger.debug("Connected to {}", new Object[] { socketChannel });
+                    } catch (final IOException e) {
+                        if (!stopped) {
+                            logger.error("{} unable to accept connection from remote peer due to {}", this, e.toString());
+                            if (logger.isDebugEnabled()) {
+                                logger.error("", e);
+                            }
+                        }
+                        return;
+                    }
+
+                    final Runnable processInputRunnable = new Runnable() {
+                        @Override
+                        public void run() {
+                            final InputStream rawInputStream;
+                            final OutputStream rawOutputStream;
+                            final String peer = socketChannel.socket().getInetAddress().getHostName();
+
+                            try {
+                                if (sslContext == null) {
+                                    rawInputStream = new SocketChannelInputStream(socketChannel);
+                                    rawOutputStream = new SocketChannelOutputStream(socketChannel);
+                                } else {
+                                    final SSLSocketChannel sslSocketChannel = new SSLSocketChannel(sslContext, socketChannel, false);
+                                    sslSocketChannel.connect();
+                                    rawInputStream = new SSLSocketChannelInputStream(sslSocketChannel);
+                                    rawOutputStream = new SSLSocketChannelOutputStream(sslSocketChannel);
+                                }
+                            } catch (IOException e) {
+                                logger.error("Cannot create input and/or output streams for {}", new Object[] { identifier }, e);
+                                if (logger.isDebugEnabled()) {
+                                    logger.error("", e);
+                                }
+                                try {
+                                    socketChannel.close();
+                                } catch (IOException swallow) {
+                                }
+                               
+                                return;
+                            }
+                            try (final InputStream in = new BufferedInputStream(rawInputStream);
+                                    final OutputStream out = new BufferedOutputStream(rawOutputStream)) {
+
+                                final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(1);
+
+                                ProtocolHandshake.receiveHandshake(in, out, versionNegotiator);
+
+                                boolean continueComms = true;
+                                while (continueComms) {
+                                    continueComms = listen(in, out, versionNegotiator.getVersion());
+                                }
+                                // client has issued 'close'
+                                logger.debug("Client issued close on {}", new Object[] { socketChannel });
+                            } catch (final SocketTimeoutException e) {
+                                logger.debug("30 sec timeout reached", e);
+                            } catch (final IOException | HandshakeException e) {
+                                if (!stopped) {
+                                    logger.error("{} unable to communicate with remote peer {} due to {}", new Object[] { this, peer, e.toString() });
+                                    if (logger.isDebugEnabled()) {
+                                        logger.error("", e);
+                                    }
+                                }
+                            } finally {
+                                processInputThreads.remove(Thread.currentThread());
+                            }
+                        }
+                    };
+
+                    final Thread processInputThread = new Thread(processInputRunnable);
+                    processInputThread.setName("Distributed Cache Server Communications Thread: " + identifier);
+                    processInputThread.setDaemon(true);
+                    processInputThread.start();
+                    processInputThreads.add(processInputThread);
+                }
+            }
+        };
+
+        final Thread thread = new Thread(runnable);
+        thread.setDaemon(true);
+        thread.setName("Distributed Cache Server: " + identifier);
+        thread.start();
+    }
+
+    @Override
+    public void stop() throws IOException {
+        stopped = true;
+        logger.info("Stopping CacheServer {}", new Object[] { this.identifier });
+
+        if (serverSocketChannel != null) {
+            serverSocketChannel.close();
+        }
+        // need to close out the created SocketChannels...this is done by interrupting
+        // the created threads that loop on listen().
+        for (Thread processInputThread : processInputThreads) {
+            processInputThread.interrupt();
+            int i = 0;
+            while (!processInputThread.isInterrupted() && i++ < 5) {
+                try {
+                    Thread.sleep(50); // allow thread to gracefully terminate
+                } catch (InterruptedException e) {
+                }
+            }
+        }
+        processInputThreads.clear();
+    }
+
+    @Override
+    public String toString() {
+        return "CacheServer[id=" + identifier + "]";
+    }
+
+    /**
+     * Listens for incoming data and communicates with remote peer
+     * 
+     * @param in
+     * @param out
+     * @param version
+     * @return <code>true</code> if communications should continue, <code>false</code> otherwise
+     * @throws IOException
+     */
+    protected abstract boolean listen(InputStream in, OutputStream out, int version) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheRecord.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheRecord.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheRecord.java
new file mode 100644
index 0000000..71ac56d
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheRecord.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class CacheRecord {
+
+    private static final AtomicLong idGenerator = new AtomicLong(0L);
+    
+    private final long id;
+    private final long entryDate;
+    private volatile long lastHitDate;
+    private final AtomicInteger hitCount = new AtomicInteger(0);
+    
+    public CacheRecord() {
+        entryDate = System.currentTimeMillis();
+        lastHitDate = entryDate;
+        id = idGenerator.getAndIncrement();
+    }
+    
+    public long getEntryDate() {
+        return entryDate;
+    }
+    
+    public long getLastHitDate() {
+        return lastHitDate;
+    }
+    
+    public int getHitCount() {
+        return hitCount.get();
+    }
+
+    public void hit() {
+        hitCount.getAndIncrement();
+        lastHitDate = System.currentTimeMillis();
+    }
+    
+    public long getId() {
+        return id;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheServer.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheServer.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheServer.java
new file mode 100644
index 0000000..2c85cd8
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheServer.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server;
+
+import java.io.IOException;
+
+public interface CacheServer {
+
+    void start() throws IOException;
+    void stop() throws IOException;
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java
new file mode 100644
index 0000000..0f962d0
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.annotation.OnConfigured;
+import org.apache.nifi.processor.annotation.OnShutdown;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.ssl.SSLContextService;
+
+public abstract class DistributedCacheServer extends AbstractControllerService {
+    public static final String EVICTION_STRATEGY_LFU = "Least Frequently Used";
+    public static final String EVICTION_STRATEGY_LRU = "Least Recently Used";
+    public static final String EVICTION_STRATEGY_FIFO = "First In, First Out";
+
+    public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+            .name("Port")
+            .description("The port to listen on for incoming connections")
+            .required(true)
+            .addValidator(StandardValidators.PORT_VALIDATOR)
+            .defaultValue("4557")
+            .build();
+    public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
+            .name("SSL Context Service")
+            .description(
+                    "If specified, this service will be used to create an SSL Context that will be used to secure communications; if not specified, communications will not be secure")
+            .required(false)
+            .addValidator(StandardValidators.createControllerServiceExistsValidator(SSLContextService.class))
+            .build();
+    public static final PropertyDescriptor MAX_CACHE_ENTRIES = new PropertyDescriptor.Builder()
+            .name("Maximum Cache Entries")
+            .description("The maximum number of cache entries that the cache can hold")
+            .required(true)
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .defaultValue("10000")
+            .build();
+    public static final PropertyDescriptor EVICTION_POLICY = new PropertyDescriptor.Builder()
+            .name("Eviction Strategy")
+            .description("Determines which strategy should be used to evict values from the cache to make room for new entries")
+            .required(true)
+            .allowableValues(EVICTION_STRATEGY_LFU, EVICTION_STRATEGY_LRU, EVICTION_STRATEGY_FIFO)
+            .defaultValue(EVICTION_STRATEGY_LFU)
+            .build();
+    public static final PropertyDescriptor PERSISTENCE_PATH = new PropertyDescriptor.Builder()
+            .name("Persistence Directory")
+            .description("If specified, the cache will be persisted in the given directory; if not specified, the cache will be in-memory only")
+            .required(false)
+            .addValidator(StandardValidators.createDirectoryExistsValidator(true, true))
+            .build();
+
+    private volatile CacheServer cacheServer;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(PORT);
+        properties.add(MAX_CACHE_ENTRIES);
+        properties.add(EVICTION_POLICY);
+        properties.add(PERSISTENCE_PATH);
+        properties.add(new PropertyDescriptor.Builder().fromPropertyDescriptor(SSL_CONTEXT_SERVICE).allowableValues(
+                getControllerServiceLookup().getControllerServiceIdentifiers(SSLContextService.class)).build());
+        return properties;
+    }
+
+    @OnConfigured
+    public void startServer(final ConfigurationContext context) throws IOException {
+        if (cacheServer == null) {
+            cacheServer = createCacheServer(context);
+            cacheServer.start();
+        }
+    }
+
+    @OnShutdown
+    public void shutdownServer() throws IOException {
+        if (cacheServer != null) {
+            cacheServer.stop();
+        }
+        cacheServer = null;
+    }
+
+    @Override
+    protected void finalize() throws Throwable {
+        shutdownServer();
+    }
+
+    protected abstract CacheServer createCacheServer(ConfigurationContext context);
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedSetCacheServer.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedSetCacheServer.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedSetCacheServer.java
new file mode 100644
index 0000000..426573f
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedSetCacheServer.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server;
+
+import java.io.File;
+
+import javax.net.ssl.SSLContext;
+
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.ssl.SSLContextService;
+import org.apache.nifi.ssl.SSLContextService.ClientAuth;
+
+public class DistributedSetCacheServer extends DistributedCacheServer {
+
+    @Override
+    protected CacheServer createCacheServer(final ConfigurationContext context) {
+        final int port = context.getProperty(PORT).asInteger();
+        final String persistencePath = context.getProperty(PERSISTENCE_PATH).getValue();
+        final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+        final int maxSize = context.getProperty(MAX_CACHE_ENTRIES).asInteger();
+        final String evictionPolicyName = context.getProperty(EVICTION_POLICY).getValue();
+        
+        final SSLContext sslContext;
+        if ( sslContextService == null ) {
+            sslContext = null;
+        } else {
+            sslContext = sslContextService.createSSLContext(ClientAuth.REQUIRED);
+        }
+        
+        final EvictionPolicy evictionPolicy;
+        switch (evictionPolicyName) {
+            case EVICTION_STRATEGY_FIFO:
+                evictionPolicy = EvictionPolicy.FIFO;
+                break;
+            case EVICTION_STRATEGY_LFU:
+                evictionPolicy = EvictionPolicy.LFU;
+                break;
+            case EVICTION_STRATEGY_LRU:
+                evictionPolicy = EvictionPolicy.LRU;
+                break;
+            default:
+                throw new IllegalArgumentException("Illegal Eviction Policy: " + evictionPolicyName);
+        }
+        
+        try {
+            final File persistenceDir = persistencePath == null ? null : new File(persistencePath);
+            
+            return new SetCacheServer(getIdentifier(), sslContext, port, maxSize, evictionPolicy, persistenceDir);
+        } catch (final Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/EvictionPolicy.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/EvictionPolicy.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/EvictionPolicy.java
new file mode 100644
index 0000000..60bd2c1
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/EvictionPolicy.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server;
+
+import java.util.Comparator;
+
+public enum EvictionPolicy {
+    LFU(new LFUComparator()),
+    LRU(new LRUComparator()),
+    FIFO(new FIFOComparator());
+    
+    private final Comparator<CacheRecord> comparator;
+    
+    private EvictionPolicy(final Comparator<CacheRecord> comparator) {
+        this.comparator = comparator;
+    }
+    
+    public Comparator<CacheRecord> getComparator() {
+        return comparator;
+    }
+    
+    public static class LFUComparator implements Comparator<CacheRecord> {
+        @Override
+        public int compare(final CacheRecord o1, final CacheRecord o2) {
+            if ( o1.equals(o2) ) {
+                return 0;
+            }
+            
+            final int hitCountComparison = Integer.compare(o1.getHitCount(), o2.getHitCount());
+            final int entryDateComparison = (hitCountComparison == 0) ? Long.compare(o1.getEntryDate(), o2.getEntryDate()) : hitCountComparison;
+            return (entryDateComparison == 0 ? Long.compare(o1.getId(), o2.getId()) : entryDateComparison);
+        }
+    }
+    
+    public static class LRUComparator implements Comparator<CacheRecord> {
+        @Override
+        public int compare(final CacheRecord o1, final CacheRecord o2) {
+            if ( o1.equals(o2) ) {
+                return 0;
+            }
+
+            final int lastHitDateComparison = Long.compare(o1.getLastHitDate(), o2.getLastHitDate());
+            return (lastHitDateComparison == 0 ? Long.compare(o1.getId(), o2.getId()) : lastHitDateComparison);
+        }
+    }
+    
+    public static class FIFOComparator implements Comparator<CacheRecord> {
+        @Override
+        public int compare(final CacheRecord o1, final CacheRecord o2) {
+            if ( o1.equals(o2) ) {
+                return 0;
+            }
+
+            final int entryDateComparison = Long.compare(o1.getEntryDate(), o2.getEntryDate());
+            return (entryDateComparison == 0 ? Long.compare(o1.getId(), o2.getId()) : entryDateComparison);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/SetCacheServer.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/SetCacheServer.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/SetCacheServer.java
new file mode 100644
index 0000000..5d2c0f6
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/SetCacheServer.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server;
+
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import javax.net.ssl.SSLContext;
+
+import org.apache.nifi.distributed.cache.server.set.PersistentSetCache;
+import org.apache.nifi.distributed.cache.server.set.SetCache;
+import org.apache.nifi.distributed.cache.server.set.SetCacheResult;
+import org.apache.nifi.distributed.cache.server.set.SimpleSetCache;
+import org.apache.nifi.io.DataOutputStream;
+
+public class SetCacheServer extends AbstractCacheServer {
+
+    private final SetCache cache;
+
+    public SetCacheServer(final String identifier, final SSLContext sslContext, final int port, final int maxSize,
+            final EvictionPolicy evictionPolicy, final File persistencePath) throws IOException {
+        super(identifier, sslContext, port);
+
+        final SetCache simpleCache = new SimpleSetCache(identifier, maxSize, evictionPolicy);
+
+        if (persistencePath == null) {
+            this.cache = simpleCache;
+        } else {
+            final PersistentSetCache persistentCache = new PersistentSetCache(identifier, persistencePath, simpleCache);
+            persistentCache.restore();
+            this.cache = persistentCache;
+        }
+    }
+
+    @Override
+    protected boolean listen(final InputStream in, final OutputStream out, final int version) throws IOException {
+        final DataInputStream dis = new DataInputStream(in);
+        final DataOutputStream dos = new DataOutputStream(out);
+
+        final String action = dis.readUTF();
+        if (action.equals("close")) {
+            return false;
+        }
+
+        final int valueLength = dis.readInt();
+        final byte[] value = new byte[valueLength];
+        dis.readFully(value);
+        final ByteBuffer valueBuffer = ByteBuffer.wrap(value);
+
+        final SetCacheResult response;
+        switch (action) {
+        case "addIfAbsent":
+            response = cache.addIfAbsent(valueBuffer);
+            break;
+        case "contains":
+            response = cache.contains(valueBuffer);
+            break;
+        case "remove":
+            response = cache.remove(valueBuffer);
+            break;
+        default:
+            throw new IOException("IllegalRequest");
+        }
+
+        dos.writeBoolean(response.getResult());
+        dos.flush();
+
+        return true;
+    }
+
+    @Override
+    public void stop() throws IOException {
+        try {
+            super.stop();
+        } finally {
+            cache.shutdown();
+        }
+    }
+
+    @Override
+    protected void finalize() throws Throwable {
+        if (!stopped)
+            stop();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/DistributedMapCacheServer.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/DistributedMapCacheServer.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/DistributedMapCacheServer.java
new file mode 100644
index 0000000..920529d
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/DistributedMapCacheServer.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server.map;
+
+import java.io.File;
+
+import javax.net.ssl.SSLContext;
+
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.distributed.cache.server.CacheServer;
+import org.apache.nifi.distributed.cache.server.DistributedCacheServer;
+import org.apache.nifi.distributed.cache.server.EvictionPolicy;
+import org.apache.nifi.ssl.SSLContextService;
+import org.apache.nifi.ssl.SSLContextService.ClientAuth;
+
+public class DistributedMapCacheServer extends DistributedCacheServer {
+
+    @Override
+    protected CacheServer createCacheServer(final ConfigurationContext context) {
+        final int port = context.getProperty(PORT).asInteger();
+        final String persistencePath = context.getProperty(PERSISTENCE_PATH).getValue();
+        final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+        final int maxSize = context.getProperty(MAX_CACHE_ENTRIES).asInteger();
+        final String evictionPolicyName = context.getProperty(EVICTION_POLICY).getValue();
+        
+        final SSLContext sslContext;
+        if ( sslContextService == null ) {
+            sslContext = null;
+        } else {
+            sslContext = sslContextService.createSSLContext(ClientAuth.REQUIRED);
+        }
+        
+        final EvictionPolicy evictionPolicy;
+        switch (evictionPolicyName) {
+            case EVICTION_STRATEGY_FIFO:
+                evictionPolicy = EvictionPolicy.FIFO;
+                break;
+            case EVICTION_STRATEGY_LFU:
+                evictionPolicy = EvictionPolicy.LFU;
+                break;
+            case EVICTION_STRATEGY_LRU:
+                evictionPolicy = EvictionPolicy.LRU;
+                break;
+            default:
+                throw new IllegalArgumentException("Illegal Eviction Policy: " + evictionPolicyName);
+        }
+        
+        try {
+            final File persistenceDir = persistencePath == null ? null : new File(persistencePath);
+            
+            return new MapCacheServer(getIdentifier(), sslContext, port, maxSize, evictionPolicy, persistenceDir);
+        } catch (final Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCache.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCache.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCache.java
new file mode 100644
index 0000000..534cb0b
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCache.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server.map;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public interface MapCache {
+
+    MapPutResult putIfAbsent(ByteBuffer key, ByteBuffer value) throws IOException;
+    boolean containsKey(ByteBuffer key) throws IOException;
+    ByteBuffer get(ByteBuffer key) throws IOException;
+    ByteBuffer remove(ByteBuffer key) throws IOException;
+    void shutdown() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheRecord.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheRecord.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheRecord.java
new file mode 100644
index 0000000..b0ab0c4
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheRecord.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server.map;
+
+import java.nio.ByteBuffer;
+
+import org.apache.nifi.distributed.cache.server.CacheRecord;
+
+public class MapCacheRecord extends CacheRecord {
+    private final ByteBuffer key;
+    private final ByteBuffer value;
+    
+    public MapCacheRecord(final ByteBuffer key, final ByteBuffer value) {
+        this.key = key;
+        this.value = value;
+    }
+    
+    public ByteBuffer getKey() {
+        return key;
+    }
+    
+    public ByteBuffer getValue() {
+        return value;
+    }
+    
+    @Override
+    public int hashCode() {
+        return 2938476 + key.hashCode() * value.hashCode();
+    }
+    
+    @Override
+    public boolean equals(final Object obj) {
+        if ( obj == this ) {
+            return true;
+        }
+        
+        if ( obj instanceof MapCacheRecord ) {
+            final MapCacheRecord that = ((MapCacheRecord) obj);
+            return key.equals(that.key) && value.equals(that.value);
+        }
+        
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheServer.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheServer.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheServer.java
new file mode 100644
index 0000000..3e8dd0e
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheServer.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server.map;
+
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import javax.net.ssl.SSLContext;
+
+import org.apache.nifi.distributed.cache.server.AbstractCacheServer;
+import org.apache.nifi.distributed.cache.server.EvictionPolicy;
+import org.apache.nifi.io.DataOutputStream;
+
+public class MapCacheServer extends AbstractCacheServer {
+
+    private final MapCache cache;
+
+    public MapCacheServer(final String identifier, final SSLContext sslContext, final int port, final int maxSize,
+            final EvictionPolicy evictionPolicy, final File persistencePath) throws IOException {
+        super(identifier, sslContext, port);
+
+        final MapCache simpleCache = new SimpleMapCache(identifier, maxSize, evictionPolicy);
+
+        if (persistencePath == null) {
+            this.cache = simpleCache;
+        } else {
+            final PersistentMapCache persistentCache = new PersistentMapCache(identifier, persistencePath, simpleCache);
+            persistentCache.restore();
+            this.cache = persistentCache;
+        }
+    }
+
+    @Override
+    protected boolean listen(final InputStream in, final OutputStream out, final int version) throws IOException {
+        final DataInputStream dis = new DataInputStream(in);
+        final DataOutputStream dos = new DataOutputStream(out);
+        final String action = dis.readUTF();
+        try {
+            switch (action) {
+            case "close": {
+                return false;
+            }
+            case "putIfAbsent": {
+                final byte[] key = readValue(dis);
+                final byte[] value = readValue(dis);
+                final MapPutResult putResult = cache.putIfAbsent(ByteBuffer.wrap(key), ByteBuffer.wrap(value));
+                dos.writeBoolean(putResult.isSuccessful());
+                break;
+            }
+            case "containsKey": {
+                final byte[] key = readValue(dis);
+                final boolean contains = cache.containsKey(ByteBuffer.wrap(key));
+                dos.writeBoolean(contains);
+                break;
+            }
+            case "getAndPutIfAbsent": {
+                final byte[] key = readValue(dis);
+                final byte[] value = readValue(dis);
+
+                final MapPutResult putResult = cache.putIfAbsent(ByteBuffer.wrap(key), ByteBuffer.wrap(value));
+                if (putResult.isSuccessful()) {
+                    // Put was successful. There was no old value to get.
+                    dos.writeInt(0);
+                } else {
+                    // we didn't put. Write back the previous value
+                    final byte[] byteArray = putResult.getExistingValue().array();
+                    dos.writeInt(byteArray.length);
+                    dos.write(byteArray);
+                }
+
+                break;
+            }
+            case "get": {
+                final byte[] key = readValue(dis);
+                final ByteBuffer existingValue = cache.get(ByteBuffer.wrap(key));
+                if (existingValue == null) {
+                    // there was no existing value; we did a "put".
+                    dos.writeInt(0);
+                } else {
+                    // a value already existed. we did not update the map
+                    final byte[] byteArray = existingValue.array();
+                    dos.writeInt(byteArray.length);
+                    dos.write(byteArray);
+                }
+
+                break;
+            }
+            case "remove": {
+                final byte[] key = readValue(dis);
+                final boolean removed = cache.remove(ByteBuffer.wrap(key)) != null;
+                dos.writeBoolean(removed);
+                break;
+            }
+            default: {
+                throw new IOException("Illegal Request");
+            }
+            }
+        } finally {
+            dos.flush();
+        }
+
+        return true;
+    }
+
+    @Override
+    public void stop() throws IOException {
+        try {
+            super.stop();
+        } finally {
+            cache.shutdown();
+        }
+    }
+
+    @Override
+    protected void finalize() throws Throwable {
+        if (!stopped)
+            stop();
+    }
+
+    private byte[] readValue(final DataInputStream dis) throws IOException {
+        final int numBytes = dis.readInt();
+        final byte[] buffer = new byte[numBytes];
+        dis.readFully(buffer);
+        return buffer;
+    }
+
+}


[50/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
new file mode 100644
index 0000000..21c67f2
--- /dev/null
+++ b/.gitignore
@@ -0,0 +1,9 @@
+target
+.project
+.settings
+nbactions.xml
+
+# Intellij
+.idea/
+*.iml
+*.iws

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/DISCLAIMER
----------------------------------------------------------------------
diff --git a/DISCLAIMER b/DISCLAIMER
new file mode 100644
index 0000000..0f8e7a1
--- /dev/null
+++ b/DISCLAIMER
@@ -0,0 +1,15 @@
+Apache NiFi is an effort undergoing incubation at the Apache Software
+Foundation (ASF), sponsored by the Apache Incubator PMC.
+
+Incubation is required of all newly accepted projects until a further review
+indicates that the infrastructure, communications, and decision making process
+have stabilized in a manner consistent with other successful ASF projects.
+
+While incubation status is not necessarily a reflection of the completeness
+or stability of the code, it does indicate that the project has yet to be
+fully endorsed by the ASF.
+
+For more information about the incubation status of the Apache NiFi project
+you can go to the following page:
+
+http://nifi.incubator.apache.org/

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/LICENSE
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
new file mode 100644
index 0000000..e02ae44
--- /dev/null
+++ b/NOTICE
@@ -0,0 +1,7 @@
+Apache NiFi
+Copyright 2014 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+Please see LICENSE for additional copyright and licensing information.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
new file mode 100644
index 0000000..3f05134
--- /dev/null
+++ b/README.md
@@ -0,0 +1,65 @@
+# Apache NiFi
+
+Apache NiFi is a dataflow system based on the concepts of flow-based programming. It is currently apart of the Apache Incubator.
+
+## Table of Contents
+
+- [Features](#features)
+- [Getting Started](#getting-started)
+- [Getting Help](#getting-help)
+- [Requirements](#requirements)
+- [License](#license)
+- [Disclaimer](#disclaimer)
+
+## Features
+
+Apache NiFi supports powerful and scalable directed graphs of data routing, transformation, and system mediation logic. Some of the high-level capabilities and objectives of Apache NiFi include:
+
+- Web-based user interface for seamless experience between design, control, feedback, and monitoring of data flows
+- Highly configurable along several dimensions of quality of service such as loss tolerant versus guaranteed delivery, low latency versus high throughput, and priority based queuing
+- Fine-grained data provenance for all data received, forked, joined, cloned, modified, sent, and ultimately dropped as data reaches its configured end-state
+- Component-based extension model along well defined interfaces enabling rapid development and effective testing 
+
+## Getting Started
+
+Execute <nifi install dir>/bin/nifi.sh
+
+## Getting Help
+If you have questions, you can reach out to our mailing list: dev@nifi.incubator.apache.org
+([archive](http://mail-archives.apache.org/mod_mbox/incubator-nifi-dev)).
+We're also often available in IRC: #nifi on
+[irc.freenode.net](http://webchat.freenode.net/?channels=#nifi).
+
+
+## Requirements
+* JDK 1.7 or higher
+
+## License
+
+Except as otherwise noted this software is licensed under the
+[Apache License, Version 2.0](http://www.apache.org/licenses/LICENSE-2.0.html)
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+## Disclaimer
+
+Apache NiFi is an effort undergoing incubation at the Apache Software
+Foundation (ASF), sponsored by the Apache Incubator PMC.
+
+Incubation is required of all newly accepted projects until a further review
+indicates that the infrastructure, communications, and decision making process
+have stabilized in a manner consistent with other successful ASF projects.
+
+While incubation status is not necessarily a reflection of the completeness
+or stability of the code, it does indicate that the project has yet to be
+fully endorsed by the ASF.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/assemblies/nifi/pom.xml
----------------------------------------------------------------------
diff --git a/assemblies/nifi/pom.xml b/assemblies/nifi/pom.xml
new file mode 100644
index 0000000..ec06a23
--- /dev/null
+++ b/assemblies/nifi/pom.xml
@@ -0,0 +1,485 @@
+<?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 for the Main Standard Release of NiFi</description>
+    
+    <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>
+                </executions>
+            </plugin>
+            <plugin>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <descriptors>
+                        <descriptor>src/main/assembly/dependencies.xml</descriptor>
+                    </descriptors>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>exec-maven-plugin</artifactId>
+                <version>1.3.2</version>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>exec</goal>
+                        </goals>
+                    </execution>
+                </executions>
+                <configuration>
+                    <executable>${project.build.directory}/${project.artifactId}-${project.version}-${nifi.assembly.id}.dir/${nifi.executable}</executable>
+                </configuration>
+            </plugin>
+        </plugins>
+        <resources>
+            <resource>
+                <directory>${basedir}/src/main/resources</directory>
+            </resource>
+            <resource>
+                <directory>${project.build.directory}/generated-resources</directory>
+                <filtering>true</filtering>
+            </resource>
+        </resources>
+    </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>
+            <version>${org.slf4j.version}</version>
+            <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>
+            <version>0.0.1-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-runtime</artifactId>
+            <version>${framework.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-resources</artifactId>
+            <version>${framework.version}</version>
+            <classifier>resources</classifier>
+            <scope>provided</scope> <!-- Provided - we don't want the zip in the libs -->
+            <type>zip</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-nar</artifactId>
+            <version>${framework.version}</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>volatile-provenance-repository-nar</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>persistent-provenance-repository-nar</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+		
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>standard-services-api-nar</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>ssl-context-service-nar</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>distributed-cache-services-nar</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-nar</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-jetty-bundle</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>update-attribute-nar</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+		
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>monitor-threshold-nar</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>hadoop-libraries-nar</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>hadoop-nar</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <!--        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>execute-script-nar</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>-->
+    </dependencies>
+    
+    <properties>
+        <framework.version>0.0.1-SNAPSHOT</framework.version>
+        
+        <!--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.storage.directory>./flowfile_repository/swap</nifi.swap.storage.directory>
+        <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.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 to execute nifi using Maven.  It is assumed that the assembly has a
+            format of type 'dir' so that the executable can be referenced in a defined location.
+        -->
+        <nifi.assembly.id>bin</nifi.assembly.id>
+        <nifi.executable>${project.artifactId}-${project.version}/bin/${project.artifactId}</nifi.executable>
+        
+    </properties>
+    <profiles>
+        <profile>
+            <id>standard-rpm</id>
+            <activation>
+                <activeByDefault>false</activeByDefault>
+            </activation>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>rpm-maven-plugin</artifactId>
+                        <version>2.0.1</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>rpm</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                        <configuration>
+                            <name>${project.artifactId}</name>
+                            <projversion>${project.version}</projversion>
+                            <summary>NiFi data flow framework</summary>
+                            <description>
+                                Nifi is a simple event processing framework largely
+                                based on the Flow-Based Programming concepts.
+                            </description>
+                            <copyright>NONE</copyright>
+                            <url>http://nifi.apache.org</url>
+                            <group>Utilities</group>
+                            <requires>
+                                <require>jdk</require>
+                            </requires>
+                            <prefix>/opt/${project.artifactId}</prefix>
+                            <defineStatements>
+                                <defineStatement>_use_internal_dependency_generator 0</defineStatement>
+                            </defineStatements>
+                            <defaultDirmode>750</defaultDirmode>
+                            <defaultFilemode>640</defaultFilemode>
+                            <defaultUsername>dataflow</defaultUsername>
+                            <defaultGroupname>dataflow</defaultGroupname>
+                            <mappings>
+                                <mapping>
+                                    <directory>/opt/${project.artifactId}/${project.artifactId}-${project.version}</directory>
+                                </mapping>
+                                <mapping>
+                                    <directory>/opt/${project.artifactId}/${project.artifactId}-${project.version}/bin</directory>
+                                    <filemode>750</filemode>
+                                    <sources>
+                                        <source>
+                                            <location>${project.build.directory}/generated-resources/bin/nifi</location>
+                                            <destination>${project.artifactId}</destination>
+                                            <filter>true</filter>
+                                        </source>
+                                        <source>
+                                            <location>${project.build.directory}/generated-resources/bin/wrapper-linux-x86-32</location>
+                                        </source>
+                                        <source>
+                                            <location>${project.build.directory}/generated-resources/bin/wrapper-linux-x86-64</location>
+                                        </source>
+                                    </sources>
+                                </mapping>
+                                <mapping>
+                                    <directory>/opt/${project.artifactId}/${project.artifactId}-${project.version}/conf</directory>
+                                    <configuration>true</configuration>
+                                    <sources>
+                                        <source>
+                                            <location>${project.build.directory}/generated-resources/conf</location>
+                                            <excludes>
+                                                <exclude>nifi.properties</exclude>
+                                            </excludes>
+                                            <filter>true</filter>
+                                        </source>
+                                        <source>
+                                            <location>${project.build.directory}/generated-resources/conf/nifi.properties</location>
+                                            <destination>${project.artifactId}.properties</destination>
+                                            <filter>true</filter>
+                                        </source>
+                                    </sources>
+                                </mapping>
+                                <mapping>
+                                    <directory>/opt/${project.artifactId}/${project.artifactId}-${project.version}/lib</directory>
+                                    <dependency />
+                                    <sources>
+                                        <source>
+                                            <location>${project.build.directory}/generated-resources/lib</location>
+                                            <excludes>
+                                                <exclude>**/*-aix*</exclude>
+                                                <exclude>**/*-freebsd*</exclude>
+                                                <exclude>**/*-hpux*</exclude>                                                
+                                                <exclude>**/*-linux-ia*</exclude>
+                                                <exclude>**/*-linux-ppc*</exclude>
+                                                <exclude>**/*-macosx*</exclude>
+                                                <exclude>**/*-solaris*</exclude>
+                                                <exclude>**/*-windows*</exclude>
+                                            </excludes>
+                                        </source>
+                                    </sources>
+                                </mapping>
+                                <mapping>
+                                    <directory>/opt/${project.artifactId}/${project.artifactId}-${project.version}/docs</directory>
+                                    <sources>
+                                        <source>
+                                            <location>${project.build.directory}/generated-resources/docs</location>
+                                        </source>
+                                    </sources>
+                                </mapping>
+                                <mapping>
+                                    <directory>/opt/${project.artifactId}/${project.artifactId}-${project.version}/logs</directory>
+                                    <sources>
+                                        <source>
+                                            <location>${project.build.directory}/generated-resources/logs</location>
+                                        </source>
+                                    </sources>
+                                </mapping>
+                            </mappings>
+                        </configuration>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/assemblies/nifi/src/main/assembly/dependencies.xml
----------------------------------------------------------------------
diff --git a/assemblies/nifi/src/main/assembly/dependencies.xml b/assemblies/nifi/src/main/assembly/dependencies.xml
new file mode 100644
index 0000000..339275b
--- /dev/null
+++ b/assemblies/nifi/src/main/assembly/dependencies.xml
@@ -0,0 +1,123 @@
+<?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>tar.gz</format>
+    </formats>
+    <includeBaseDirectory>true</includeBaseDirectory>
+    <baseDirectory>${project.artifactId}-${project.version}</baseDirectory>
+
+    <dependencySets>
+        <dependencySet>
+            <scope>runtime</scope>
+            <outputDirectory>lib</outputDirectory>
+            <useTransitiveFiltering>true</useTransitiveFiltering>
+        </dependencySet>
+    </dependencySets>
+
+    <fileSets>
+        <fileSet>
+            <directory>${project.build.directory}/generated-resources/bin</directory>
+            <outputDirectory>bin</outputDirectory>
+            <directoryMode>0750</directoryMode>
+            <fileMode>0750</fileMode>
+            <filtered>false</filtered>
+        </fileSet>
+        <fileSet>
+            <directory>${project.build.directory}/generated-resources/conf</directory>
+            <outputDirectory>conf</outputDirectory>
+            <excludes>
+                <exclude>nifi.properties</exclude>
+            </excludes>
+            <directoryMode>0750</directoryMode>
+            <fileMode>0640</fileMode>
+            <filtered>true</filtered>
+        </fileSet>
+        <fileSet>
+            <directory>${project.build.directory}/generated-resources/lib</directory>
+            <outputDirectory>lib</outputDirectory>
+            <directoryMode>0750</directoryMode>
+            <fileMode>0640</fileMode>
+        </fileSet>
+        <fileSet>
+            <directory>${project.build.directory}/generated-resources/logs</directory>
+            <outputDirectory>logs</outputDirectory>
+            <directoryMode>0750</directoryMode>
+            <fileMode>0640</fileMode>
+        </fileSet>
+        <fileSet>
+            <directory>${project.build.directory}/generated-resources/docs</directory>
+            <outputDirectory>docs</outputDirectory>
+            <directoryMode>0750</directoryMode>
+            <fileMode>0640</fileMode>
+            <filtered>false</filtered>
+        </fileSet>
+    </fileSets>
+
+    <files>
+        <file>
+            <source>${project.build.directory}/generated-resources/conf/nifi.properties</source>
+            <outputDirectory>conf</outputDirectory>
+            <destName>${project.artifactId}.properties</destName>
+            <fileMode>0640</fileMode>
+            <filtered>true</filtered>
+        </file>
+        <file>
+            <source>${project.build.directory}/generated-resources/bin/nifi.sh</source>
+            <outputDirectory>bin</outputDirectory>
+            <destName>nifi.sh</destName>
+            <fileMode>0750</fileMode>
+            <filtered>true</filtered>
+        </file>
+        <file>
+            <source>${project.build.directory}/generated-resources/docs/README.md</source>
+            <outputDirectory>docs</outputDirectory>
+            <destName>README.md</destName>
+            <fileMode>0640</fileMode>
+            <filtered>true</filtered>
+        </file>
+        <file>
+            <source>${project.build.directory}/generated-resources/DISCLAIMER</source>
+            <outputDirectory>./</outputDirectory>
+            <destName>DISCLAIMER</destName>
+            <fileMode>0640</fileMode>
+            <filtered>true</filtered>
+        </file>
+        <file>
+            <source>${project.build.directory}/generated-resources/DISCLAIMER</source>
+            <outputDirectory>./</outputDirectory>
+            <destName>DISCLAIMER</destName>
+            <fileMode>0640</fileMode>
+            <filtered>true</filtered>
+        </file>    
+        <file>
+            <source>${project.build.directory}/generated-resources/LICENSE</source>
+            <outputDirectory>./</outputDirectory>
+            <destName>LICENSE</destName>
+            <fileMode>0640</fileMode>
+            <filtered>true</filtered>
+        </file>       
+        <file>
+            <source>${project.build.directory}/generated-resources/NOTICE</source>
+            <outputDirectory>./</outputDirectory>
+            <destName>NOTICE</destName>
+            <fileMode>0640</fileMode>
+            <filtered>true</filtered>
+        </file>
+    </files>
+</assembly>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/core-flowfile-attributes/pom.xml
----------------------------------------------------------------------
diff --git a/commons/core-flowfile-attributes/pom.xml b/commons/core-flowfile-attributes/pom.xml
new file mode 100644
index 0000000..d611134
--- /dev/null
+++ b/commons/core-flowfile-attributes/pom.xml
@@ -0,0 +1,32 @@
+<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-parent</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-core-flowfile-attributes</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <packaging>jar</packaging>
+
+    <name>NiFi: Core FlowFile Attributes</name>
+    
+    <dependencies>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/core-flowfile-attributes/src/main/java/org/apache/nifi/flowfile/attributes/CoreAttributes.java
----------------------------------------------------------------------
diff --git a/commons/core-flowfile-attributes/src/main/java/org/apache/nifi/flowfile/attributes/CoreAttributes.java b/commons/core-flowfile-attributes/src/main/java/org/apache/nifi/flowfile/attributes/CoreAttributes.java
new file mode 100644
index 0000000..24f43ca
--- /dev/null
+++ b/commons/core-flowfile-attributes/src/main/java/org/apache/nifi/flowfile/attributes/CoreAttributes.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/core-flowfile-attributes/src/main/java/org/apache/nifi/flowfile/attributes/FlowFileAttributeKey.java
----------------------------------------------------------------------
diff --git a/commons/core-flowfile-attributes/src/main/java/org/apache/nifi/flowfile/attributes/FlowFileAttributeKey.java b/commons/core-flowfile-attributes/src/main/java/org/apache/nifi/flowfile/attributes/FlowFileAttributeKey.java
new file mode 100644
index 0000000..cc6c28e
--- /dev/null
+++ b/commons/core-flowfile-attributes/src/main/java/org/apache/nifi/flowfile/attributes/FlowFileAttributeKey.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/data-provenance-utils/.gitignore
----------------------------------------------------------------------
diff --git a/commons/data-provenance-utils/.gitignore b/commons/data-provenance-utils/.gitignore
new file mode 100755
index 0000000..19f2e00
--- /dev/null
+++ b/commons/data-provenance-utils/.gitignore
@@ -0,0 +1,2 @@
+/target
+/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/data-provenance-utils/pom.xml
----------------------------------------------------------------------
diff --git a/commons/data-provenance-utils/pom.xml b/commons/data-provenance-utils/pom.xml
new file mode 100644
index 0000000..0f33c06
--- /dev/null
+++ b/commons/data-provenance-utils/pom.xml
@@ -0,0 +1,42 @@
+<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-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>
+            <version>[0.0.1-SNAPSHOT,1.0.0-SNAPSHOT)</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-core-flowfile-attributes</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..dc24a93
--- /dev/null
+++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/AsyncLineageSubmission.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..4244476
--- /dev/null
+++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/AsyncQuerySubmission.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..dc2903f
--- /dev/null
+++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/NamedSearchableField.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..6a934b1
--- /dev/null
+++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/SearchableFieldParser.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..97c9880
--- /dev/null
+++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/SearchableFields.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+    }
+}


[02/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..1443737
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
@@ -0,0 +1,875 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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();
+        }
+    }
+}


[45/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..6b7b838
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ModEvaluator.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..a3b03ed
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MultiplyEvaluator.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..327e620
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotEvaluator.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..126fbfa
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotNullEvaluator.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..bfde6b0
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NowEvaluator.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..672d855
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NumberToDateEvaluator.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..11d3fa9
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OneUpSequenceEvaluator.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..ab3a16c
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OrEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..dff693d
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..5b66b8f
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..19ed63e
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..d6a7c5b
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEvaluator.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..91333b7
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceNullEvaluator.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..c6a7eb7
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StartsWithEvaluator.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..a4e21a4
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StringToDateEvaluator.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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).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/4d998c12/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
new file mode 100644
index 0000000..a8b7357
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterEvaluator.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..3dac40e
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterLastEvaluator.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..92b136c
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeEvaluator.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..818a3dc
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeLastEvaluator.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..da11c45
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringEvaluator.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..7de01eb
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToLowerEvaluator.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1d4664a
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToNumberEvaluator.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..aab094d
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToRadixEvaluator.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..2f7fbe3
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToStringEvaluator.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..29bc61d
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToUpperEvaluator.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b9926d7
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/TrimEvaluator.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..6c3ba14
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlDecodeEvaluator.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..d1e2caa
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlEncodeEvaluator.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..faa8f0a
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UuidEvaluator.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..a085423
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/BooleanLiteralEvaluator.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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;
+    }
+
+}


[12/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..3defea7
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
@@ -0,0 +1,3616 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 (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
+    public NodeResponse applyRequest(final String method, final URI uri, final Object entity, final Map<String, String> head

<TRUNCATED>

[43/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/nifi-file-utils/src/main/java/org/apache/nifi/file/FileUtils.java
----------------------------------------------------------------------
diff --git a/commons/nifi-file-utils/src/main/java/org/apache/nifi/file/FileUtils.java b/commons/nifi-file-utils/src/main/java/org/apache/nifi/file/FileUtils.java
new file mode 100644
index 0000000..8920493
--- /dev/null
+++ b/commons/nifi-file-utils/src/main/java/org/apache/nifi/file/FileUtils.java
@@ -0,0 +1,612 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Random;
+import org.apache.commons.codec.digest.DigestUtils;
+
+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 {
+        BufferedInputStream bis = null;
+        try {
+            bis = new BufferedInputStream(new FileInputStream(file));
+            return DigestUtils.md5(bis);
+        } finally {
+            FileUtils.closeQuietly(bis);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/nifi-logging-utils/pom.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-logging-utils/pom.xml b/commons/nifi-logging-utils/pom.xml
new file mode 100644
index 0000000..ce5064b
--- /dev/null
+++ b/commons/nifi-logging-utils/pom.xml
@@ -0,0 +1,35 @@
+<?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-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/4d998c12/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
new file mode 100644
index 0000000..7c71d85
--- /dev/null
+++ b/commons/nifi-logging-utils/src/main/java/org/apache/nifi/logging/NiFiLog.java
@@ -0,0 +1,367 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/nifi-parent/pom.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-parent/pom.xml b/commons/nifi-parent/pom.xml
new file mode 100644
index 0000000..7684d53
--- /dev/null
+++ b/commons/nifi-parent/pom.xml
@@ -0,0 +1,217 @@
+<?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>
+    <groupId>org.apache.nifi</groupId>
+    <artifactId>nifi-parent</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <packaging>pom</packaging>
+    <name>NiFi Parent</name>
+
+    <description>A helpful parent pom which can be used for all NiFi components.  Helps establish the basic requirements/depdencies.</description>
+	
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <org.slf4j.version>1.7.7</org.slf4j.version>
+    </properties>
+
+    <build>
+        <pluginManagement>
+            <plugins>
+                <plugin>
+                    <groupId>org.codehaus.mojo</groupId>
+                    <artifactId>findbugs-maven-plugin</artifactId>
+                    <version>3.0.0</version>
+                    <configuration>
+                        <effort>Max</effort>
+                        <threshold>Medium</threshold>
+                        <xmlOutput>true</xmlOutput>
+                    </configuration>
+                    <executions>
+                        <execution>
+                            <goals>
+                                <goal>check</goal>
+                            </goals>
+                        </execution>
+                    </executions>
+                </plugin>
+            </plugins>
+        </pluginManagement>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nar-maven-plugin</artifactId>
+                <version>0.0.1-SNAPSHOT</version>
+                <extensions>true</extensions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <configuration>
+                    <source>1.7</source>
+                    <target>1.7</target>
+                    <fork>true</fork>
+                </configuration>
+                <version>3.2</version>
+            </plugin>
+            <plugin>
+                <artifactId>maven-jar-plugin</artifactId>
+                <version>2.5</version>
+            </plugin>
+            <plugin>
+                <artifactId>maven-war-plugin</artifactId>
+                <version>2.5</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>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <version>2.5.2</version>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-release-plugin</artifactId>
+                <version>2.5.1</version>
+            </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.codehaus.mojo</groupId>
+                <artifactId>exec-maven-plugin</artifactId>
+                <version>1.3.2</version>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-site-plugin</artifactId>
+                <version>3.4</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>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+    <dependencyManagement>
+        <dependencies>
+            <dependency>
+                <groupId>ch.qos.logback</groupId>
+                <artifactId>logback-classic</artifactId>
+                <version>1.1.2</version>
+                <scope>provided</scope>
+                <exclusions>
+                    <exclusion>
+                        <groupId>org.slf4j</groupId>
+                        <artifactId>slf4j-api</artifactId>
+                    </exclusion>
+                </exclusions>
+            </dependency>
+            <dependency>
+                <groupId>org.slf4j</groupId>
+                <artifactId>jcl-over-slf4j</artifactId>
+                <version>${org.slf4j.version}</version>
+                <scope>provided</scope>
+            </dependency>
+            <dependency>
+                <groupId>org.slf4j</groupId>
+                <artifactId>jul-to-slf4j</artifactId>
+                <version>${org.slf4j.version}</version>
+                <scope>provided</scope>
+            </dependency>
+            <dependency>
+                <groupId>org.slf4j</groupId>
+                <artifactId>slf4j-api</artifactId>
+                <version>${org.slf4j.version}</version>
+                <scope>provided</scope>
+            </dependency>
+        </dependencies>
+    </dependencyManagement>
+    <dependencies>
+        <!-- required for libraries using commons-loggings -->
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>4.11</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <version>1.10.8</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <version>${org.slf4j.version}</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+    <!-- All projects use the same distrubution Manager for publishing artifacts 
+    but for obtaining them this is specified in the settings.xml file for each 
+    user -->
+    <distributionManagement>
+        <repository>
+            <id>nifi-releases</id>
+            <url>${nifi.repo.url}</url>
+        </repository>
+        <snapshotRepository>
+            <id>nifi-snapshots</id>
+            <url>${nifi.snapshot.repo.url}</url>
+        </snapshotRepository>
+    </distributionManagement>
+    <reporting>
+        <plugins>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>findbugs-maven-plugin</artifactId>
+                <version>3.0.0</version>
+                <configuration>
+                    <effort>Max</effort>
+                    <threshold>Medium</threshold>
+                    <xmlOutput>true</xmlOutput>
+                </configuration>
+            </plugin>
+        </plugins>
+    </reporting>
+</project>

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

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/nifi-properties/pom.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-properties/pom.xml b/commons/nifi-properties/pom.xml
new file mode 100644
index 0000000..70f90aa
--- /dev/null
+++ b/commons/nifi-properties/pom.xml
@@ -0,0 +1,29 @@
+<?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-parent</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-properties</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <name>NiFi Properties</name>
+
+    <dependencies>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..1520814
--- /dev/null
+++ b/commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
@@ -0,0 +1,882 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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_STORAGE_LOCATION = "nifi.swap.storage.directory";
+    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";
+
+    // 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 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 File getSwapStorageLocation() {
+        final String location = getProperty(SWAP_STORAGE_LOCATION);
+        if (location == null) {
+            return new File(DEFAULT_SWAP_STORAGE_LOCATION);
+        } else {
+            return new File(location);
+        }
+    }
+
+    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);
+    }
+
+    public Boolean isSiteToSiteSecure() {
+        final String secureVal = getProperty(SITE_TO_SITE_SECURE);
+        if (secureVal == null) {
+            return null;
+        }
+
+        if ("true".equalsIgnoreCase(secureVal)) {
+            return true;
+        }
+        if ("false".equalsIgnoreCase(secureVal)) {
+            return false;
+        }
+
+        throw new IllegalStateException("Property value for " + SITE_TO_SITE_SECURE + " is " + secureVal + "; expected 'true' or 'false'");
+    }
+
+    /**
+     * 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 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;
+    }
+
+}


[35/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/extensions/file-authorization-provider/src/main/java/org/apache/nifi/authorization/FileAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/extensions/file-authorization-provider/src/main/java/org/apache/nifi/authorization/FileAuthorizationProvider.java b/extensions/file-authorization-provider/src/main/java/org/apache/nifi/authorization/FileAuthorizationProvider.java
new file mode 100644
index 0000000..55f59ba
--- /dev/null
+++ b/extensions/file-authorization-provider/src/main/java/org/apache/nifi/authorization/FileAuthorizationProvider.java
@@ -0,0 +1,568 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.io.IOException;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.HashSet;
+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.Marshaller;
+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.UnknownIdentityException;
+import org.apache.nifi.file.FileUtils;
+import org.apache.nifi.user.generated.ObjectFactory;
+import org.apache.nifi.user.generated.Role;
+import org.apache.nifi.user.generated.User;
+import org.apache.nifi.user.generated.Users;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.xml.sax.SAXException;
+
+/**
+ * Provides identity checks and grants authorities.
+ */
+public class FileAuthorizationProvider implements AuthorityProvider {
+
+    private static final Logger logger = LoggerFactory.getLogger(FileAuthorizationProvider.class);
+    private static final String USERS_XSD = "/users.xsd";
+    private static final String JAXB_GENERATED_PATH = "org.apache.nifi.user.generated";
+    private static final JAXBContext JAXB_CONTEXT = initializeJaxbContext();
+
+    /**
+     * Load the JAXBContext.
+     */
+    private static JAXBContext initializeJaxbContext() {
+        try {
+            return JAXBContext.newInstance(JAXB_GENERATED_PATH, FileAuthorizationProvider.class.getClassLoader());
+        } catch (JAXBException e) {
+            throw new RuntimeException("Unable to create JAXBContext.");
+        }
+    }
+
+    private NiFiProperties properties;
+    private File usersFile;
+    private File restoreUsersFile;
+    private Users users;
+    private final Set<String> defaultAuthorities = new HashSet<>();
+
+    @Override
+    public void initialize(final AuthorityProviderInitializationContext initializationContext) throws ProviderCreationException {
+    }
+
+    @Override
+    public void onConfigured(final AuthorityProviderConfigurationContext configurationContext) throws ProviderCreationException {
+        try {
+            final String usersFilePath = configurationContext.getProperty("Authorized Users File");
+            if (usersFilePath == null || usersFilePath.trim().isEmpty()) {
+                throw new ProviderCreationException("The authorized users file must be specified.");
+            }
+
+            // the users file instance will never be null because a default is used
+            usersFile = new File(usersFilePath);
+            final File usersFileDirectory = usersFile.getParentFile();
+
+            // the restore directory is optional and may be null
+            final File restoreDirectory = properties.getRestoreDirectory();
+
+            if (restoreDirectory != null) {
+
+                // 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 (usersFileDirectory.getAbsolutePath().equals(restoreDirectory.getAbsolutePath())) {
+                    throw new ProviderCreationException(String.format("Authorized User's directory '%s' is the same as restore directory '%s' ",
+                            usersFileDirectory.getAbsolutePath(), restoreDirectory.getAbsolutePath()));
+                }
+
+                // the restore copy will have same file name, but reside in a different directory
+                restoreUsersFile = new File(restoreDirectory, usersFile.getName());
+
+                // sync the primary copy with the restore copy
+                try {
+                    FileUtils.syncWithRestore(usersFile, restoreUsersFile, logger);
+                } catch (final IOException | IllegalStateException ioe) {
+                    throw new ProviderCreationException(ioe);
+                }
+
+            }
+
+            // load the users from the specified file
+            if (usersFile.exists()) {
+                // find the schema
+                final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
+                final Schema schema = schemaFactory.newSchema(FileAuthorizationProvider.class.getResource(USERS_XSD));
+
+                // attempt to unmarshal
+                final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
+                unmarshaller.setSchema(schema);
+                final JAXBElement<Users> element = unmarshaller.unmarshal(new StreamSource(usersFile), Users.class);
+                users = element.getValue();
+            } else {
+                final ObjectFactory objFactory = new ObjectFactory();
+                users = objFactory.createUsers();
+            }
+
+            // attempt to load a default roles
+            final String rawDefaultAuthorities = configurationContext.getProperty("Default User Roles");
+            if (StringUtils.isNotBlank(rawDefaultAuthorities)) {
+                final Set<String> invalidDefaultAuthorities = new HashSet<>();
+
+                // validate the specified authorities
+                final String[] rawDefaultAuthorityList = rawDefaultAuthorities.split(",");
+                for (String rawAuthority : rawDefaultAuthorityList) {
+                    rawAuthority = rawAuthority.trim();
+                    final Authority authority = Authority.valueOfAuthority(rawAuthority);
+                    if (authority == null) {
+                        invalidDefaultAuthorities.add(rawAuthority);
+                    } else {
+                        defaultAuthorities.add(rawAuthority);
+                    }
+                }
+
+                // report any unrecognized authorities
+                if (!invalidDefaultAuthorities.isEmpty()) {
+                    logger.warn(String.format("The following default role(s) '%s' were not recognized. Possible values: %s.",
+                            StringUtils.join(invalidDefaultAuthorities, ", "), StringUtils.join(Authority.getRawAuthorities(), ", ")));
+                }
+            }
+        } catch (IOException | ProviderCreationException | SAXException | JAXBException e) {
+            throw new ProviderCreationException(e);
+        }
+
+    }
+
+    @Override
+    public void preDestruction() {
+    }
+
+    /**
+     * Determines if this provider has a default role.
+     *
+     * @return
+     */
+    private boolean hasDefaultRoles() {
+        return !defaultAuthorities.isEmpty();
+    }
+
+    /**
+     * Determines if the specified dn is known to this authority provider. When
+     * this provider is configured to have default role(s), all dn are
+     * considered to exist.
+     *
+     * @param dn
+     * @return True if he dn is known, false otherwise
+     */
+    @Override
+    public boolean doesDnExist(String dn) throws AuthorityAccessException {
+        if (hasDefaultRoles()) {
+            return true;
+        }
+
+        final User user = getUser(dn);
+        return user != null;
+    }
+
+    /**
+     * Loads the authorities for the specified user. If this provider is
+     * configured for default user role(s) and a non existent dn is specified, a
+     * new user will be automatically created with the default role(s).
+     *
+     * @param dn
+     * @return
+     * @throws UnknownIdentityException
+     * @throws AuthorityAccessException
+     */
+    @Override
+    public synchronized Set<Authority> getAuthorities(String dn) throws UnknownIdentityException, AuthorityAccessException {
+        final Set<Authority> authorities = EnumSet.noneOf(Authority.class);
+
+        // get the user 
+        final User user = getUser(dn);
+
+        // ensure the user was located
+        if (user == null) {
+            if (hasDefaultRoles()) {
+                logger.debug(String.format("User DN not found: %s. Creating new user with default roles.", dn));
+
+                // create the user (which will automatically add any default authorities)
+                addUser(dn, null);
+
+                // get the authorities for the newly created user
+                authorities.addAll(getAuthorities(dn));
+            } else {
+                throw new UnknownIdentityException(String.format("User DN not found: %s.", dn));
+            }
+        } else {
+            // create the authorities that this user has
+            for (final Role role : user.getRole()) {
+                authorities.add(Authority.valueOfAuthority(role.getName()));
+            }
+        }
+
+        return authorities;
+    }
+
+    /**
+     * Adds the specified authorities to the specified user. Regardless of
+     * whether this provider is configured for a default user role, when a non
+     * existent dn is specified, an UnknownIdentityException will be thrown.
+     *
+     * @param dn
+     * @param authorities
+     * @throws UnknownIdentityException
+     * @throws AuthorityAccessException
+     */
+    @Override
+    public synchronized void setAuthorities(String dn, Set<Authority> authorities) throws UnknownIdentityException, AuthorityAccessException {
+        // get the user
+        final User user = getUser(dn);
+
+        // ensure the user was located
+        if (user == null) {
+            throw new UnknownIdentityException(String.format("User DN not found: %s.", dn));
+        }
+
+        // add the user authorities
+        setUserAuthorities(user, authorities);
+
+        try {
+            // save the file
+            save();
+        } catch (Exception e) {
+            throw new AuthorityAccessException(e.getMessage(), e);
+        }
+    }
+
+    /**
+     * Adds the specified authorities to the specified user.
+     *
+     * @param user
+     * @param authorities
+     */
+    private void setUserAuthorities(final User user, final Set<Authority> authorities) {
+        // clear the existing rules
+        user.getRole().clear();
+
+        // set the new roles
+        final ObjectFactory objFactory = new ObjectFactory();
+        for (final Authority authority : authorities) {
+            final Role role = objFactory.createRole();
+            role.setName(authority.toString());
+
+            // add the new role
+            user.getRole().add(role);
+        }
+    }
+
+    /**
+     * Adds the specified user. If this provider is configured with default
+     * role(s) they will be added to the new user.
+     *
+     * @param dn
+     * @param group
+     * @throws UnknownIdentityException
+     * @throws AuthorityAccessException
+     */
+    @Override
+    public synchronized void addUser(String dn, String group) throws IdentityAlreadyExistsException, AuthorityAccessException {
+        final User user = getUser(dn);
+
+        // ensure the user doesn't already exist
+        if (user != null) {
+            throw new IdentityAlreadyExistsException(String.format("User DN already exists: %s", dn));
+        }
+
+        // create the new user
+        final ObjectFactory objFactory = new ObjectFactory();
+        final User newUser = objFactory.createUser();
+
+        // set the user properties
+        newUser.setDn(dn);
+        newUser.setGroup(group);
+
+        // add default roles if appropriate
+        if (hasDefaultRoles()) {
+            for (final String authority : defaultAuthorities) {
+                Role role = objFactory.createRole();
+                role.setName(authority);
+
+                // add the role
+                newUser.getRole().add(role);
+            }
+        }
+
+        // add the user
+        users.getUser().add(newUser);
+
+        try {
+            // save the file
+            save();
+        } catch (Exception e) {
+            throw new AuthorityAccessException(e.getMessage(), e);
+        }
+    }
+
+    /**
+     * Gets the users for the specified authority.
+     *
+     * @param authority
+     * @return
+     * @throws AuthorityAccessException
+     */
+    @Override
+    public synchronized Set<String> getUsers(Authority authority) throws AuthorityAccessException {
+        final Set<String> userSet = new HashSet<>();
+        for (final User user : users.getUser()) {
+            for (final Role role : user.getRole()) {
+                if (role.getName().equals(authority.toString())) {
+                    userSet.add(user.getDn());
+                }
+            }
+        }
+        return userSet;
+    }
+
+    /**
+     * Removes the specified user. Regardless of whether this provider is
+     * configured for a default user role, when a non existent dn is specified,
+     * an UnknownIdentityException will be thrown.
+     *
+     * @param dn
+     * @throws UnknownIdentityException
+     * @throws AuthorityAccessException
+     */
+    @Override
+    public synchronized void revokeUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
+        // get the user
+        final User user = getUser(dn);
+
+        // ensure the user was located
+        if (user == null) {
+            throw new UnknownIdentityException(String.format("User DN not found: %s.", dn));
+        }
+
+        // remove the specified user
+        users.getUser().remove(user);
+
+        try {
+            // save the file
+            save();
+        } catch (Exception e) {
+            throw new AuthorityAccessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void setUsersGroup(Set<String> dns, String group) throws UnknownIdentityException, AuthorityAccessException {
+        final Collection<User> groupedUsers = new HashSet<>();
+
+        // get the specified users
+        for (final String dn : dns) {
+            // get the user
+            final User user = getUser(dn);
+
+            // ensure the user was located
+            if (user == null) {
+                throw new UnknownIdentityException(String.format("User DN not found: %s.", dn));
+            }
+
+            groupedUsers.add(user);
+        }
+
+        // update each user group
+        for (final User user : groupedUsers) {
+            user.setGroup(group);
+        }
+
+        try {
+            // save the file
+            save();
+        } catch (Exception e) {
+            throw new AuthorityAccessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void ungroupUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
+        // get the user
+        final User user = getUser(dn);
+
+        // ensure the user was located
+        if (user == null) {
+            throw new UnknownIdentityException(String.format("User DN not found: %s.", dn));
+        }
+
+        // remove the users group
+        user.setGroup(null);
+
+        try {
+            // save the file
+            save();
+        } catch (Exception e) {
+            throw new AuthorityAccessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void ungroup(String group) throws AuthorityAccessException {
+        // get the user group
+        final Collection<User> userGroup = getUserGroup(group);
+
+        // ensure the user group was located
+        if (userGroup == null) {
+            return;
+        }
+
+        // update each user group
+        for (final User user : userGroup) {
+            user.setGroup(null);
+        }
+
+        try {
+            // save the file
+            save();
+        } catch (Exception e) {
+            throw new AuthorityAccessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public String getGroupForUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
+        // get the user
+        final User user = getUser(dn);
+
+        // ensure the user was located
+        if (user == null) {
+            throw new UnknownIdentityException(String.format("User DN not found: %s.", dn));
+        }
+
+        return user.getGroup();
+    }
+
+    @Override
+    public void revokeGroup(String group) throws UnknownIdentityException, AuthorityAccessException {
+        // get the user group
+        final Collection<User> userGroup = getUserGroup(group);
+
+        // ensure the user group was located
+        if (userGroup == null) {
+            throw new UnknownIdentityException(String.format("User group not found: %s.", group));
+        }
+
+        // remove each user in the group
+        for (final User user : userGroup) {
+            users.getUser().remove(user);
+        }
+
+        try {
+            // save the file
+            save();
+        } catch (Exception e) {
+            throw new AuthorityAccessException(e.getMessage(), e);
+        }
+    }
+
+    /**
+     * Locates the user with the specified DN.
+     *
+     * @param dn
+     * @return
+     */
+    private User getUser(String dn) throws UnknownIdentityException {
+        // ensure the DN was specified
+        if (dn == null) {
+            throw new UnknownIdentityException("User DN not specified.");
+        }
+
+        // attempt to get the user and ensure it was located
+        User desiredUser = null;
+        for (final User user : users.getUser()) {
+            if (dn.equalsIgnoreCase(user.getDn())) {
+                desiredUser = user;
+                break;
+            }
+        }
+
+        return desiredUser;
+    }
+
+    /**
+     * Locates all users that are part of the specified group.
+     *
+     * @param group
+     * @return
+     * @throws UnknownIdentityException
+     */
+    private Collection<User> getUserGroup(String group) throws UnknownIdentityException {
+        // ensure the DN was specified
+        if (group == null) {
+            throw new UnknownIdentityException("User group not specified.");
+        }
+
+        // get all users with this group
+        Collection<User> userGroup = null;
+        for (final User user : users.getUser()) {
+            if (group.equals(user.getGroup())) {
+                if (userGroup == null) {
+                    userGroup = new HashSet<>();
+                }
+                userGroup.add(user);
+            }
+        }
+
+        return userGroup;
+    }
+
+    /**
+     * Saves the users file.
+     *
+     * @throws Exception
+     */
+    private void save() throws Exception {
+        final Marshaller marshaller = JAXB_CONTEXT.createMarshaller();
+        marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, Boolean.TRUE);
+
+        // save users to restore directory before primary directory
+        if (restoreUsersFile != null) {
+            marshaller.marshal(users, restoreUsersFile);
+        }
+
+        // save users to primary directory
+        marshaller.marshal(users, usersFile);
+    }
+
+    @AuthorityProviderContext
+    public void setNiFiProperties(NiFiProperties properties) {
+        this.properties = properties;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/extensions/file-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider
----------------------------------------------------------------------
diff --git a/extensions/file-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider b/extensions/file-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider
new file mode 100755
index 0000000..93d2941
--- /dev/null
+++ b/extensions/file-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider
@@ -0,0 +1,15 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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.authorization.FileAuthorizationProvider

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/extensions/file-authorization-provider/src/main/xsd/users.xsd
----------------------------------------------------------------------
diff --git a/extensions/file-authorization-provider/src/main/xsd/users.xsd b/extensions/file-authorization-provider/src/main/xsd/users.xsd
new file mode 100644
index 0000000..4ee1e17
--- /dev/null
+++ b/extensions/file-authorization-provider/src/main/xsd/users.xsd
@@ -0,0 +1,64 @@
+<?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="Role">
+        <xs:attribute name="name">
+            <xs:simpleType>
+                <xs:restriction base="xs:string">
+                    <xs:enumeration value="ROLE_MONITOR"/>
+                    <xs:enumeration value="ROLE_PROVENANCE"/>
+                    <xs:enumeration value="ROLE_DFM"/>
+                    <xs:enumeration value="ROLE_ADMIN"/>
+                    <xs:enumeration value="ROLE_PROXY"/>
+                    <xs:enumeration value="ROLE_NIFI"/>
+                </xs:restriction>
+            </xs:simpleType>
+        </xs:attribute>
+    </xs:complexType>
+
+    <!-- user -->
+    <xs:complexType name="User">
+        <xs:sequence>
+            <xs:element name="role" type="Role" minOccurs="0" maxOccurs="unbounded"/>
+        </xs:sequence>
+        <xs:attribute name="dn">
+            <xs:simpleType>
+                <xs:restriction base="xs:string">
+                    <xs:minLength value="1"/>
+                    <xs:pattern value=".*[^\s].*"/>
+                </xs:restriction>
+            </xs:simpleType>
+        </xs:attribute>
+        <xs:attribute name="group">
+            <xs:simpleType>
+                <xs:restriction base="xs:string">
+                    <xs:minLength value="1"/>
+                    <xs:pattern value=".*[^\s].*"/>
+                </xs:restriction>
+            </xs:simpleType>
+        </xs:attribute>
+    </xs:complexType>
+
+    <!-- users -->
+    <xs:element name="users">
+        <xs:complexType>
+            <xs:sequence>
+                <xs:element name="user" type="User" 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/4d998c12/extensions/file-authorization-provider/src/test/java/org/apache/nifi/authorization/FileAuthorizationProviderTest.java
----------------------------------------------------------------------
diff --git a/extensions/file-authorization-provider/src/test/java/org/apache/nifi/authorization/FileAuthorizationProviderTest.java b/extensions/file-authorization-provider/src/test/java/org/apache/nifi/authorization/FileAuthorizationProviderTest.java
new file mode 100644
index 0000000..3d0196d
--- /dev/null
+++ b/extensions/file-authorization-provider/src/test/java/org/apache/nifi/authorization/FileAuthorizationProviderTest.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.io.FileOutputStream;
+import java.io.IOException;
+import org.apache.nifi.authorization.exception.ProviderCreationException;
+import org.apache.nifi.file.FileUtils;
+import org.apache.nifi.util.NiFiProperties;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import static org.mockito.Mockito.*;
+import static org.junit.Assert.*;
+import org.junit.Ignore;
+import org.mockito.Mockito;
+
+@Ignore
+public class FileAuthorizationProviderTest {
+    
+    private FileAuthorizationProvider provider;
+    
+    private File primary;
+    
+    private File restore;
+    
+    private NiFiProperties mockProperties;
+    
+    private AuthorityProviderConfigurationContext mockConfigurationContext;
+    
+    @Before
+    public void setup() throws IOException {
+        
+        primary = new File("target/primary/users.txt");
+        restore = new File("target/restore/users.txt");
+        
+        System.out.println("absolute path: " + primary.getAbsolutePath());
+        
+        mockProperties = mock(NiFiProperties.class);
+        when(mockProperties.getRestoreDirectory()).thenReturn(restore.getParentFile());
+        
+        mockConfigurationContext = mock(AuthorityProviderConfigurationContext.class);
+        when(mockConfigurationContext.getProperty(Mockito.eq("Authorized Users File"))).thenReturn(primary.getPath());
+        
+        provider = new FileAuthorizationProvider();
+        provider.setNiFiProperties(mockProperties);
+        provider.initialize(null);
+    }     
+    
+    @After
+    public void cleanup() throws Exception {
+        deleteFile(primary);
+        deleteFile(restore);
+    }
+    
+    private boolean deleteFile(final File file) {
+        if(file.isDirectory()) {
+            FileUtils.deleteFilesInDir(file, null, null, true, true);
+        }
+        return FileUtils.deleteFile(file, null, 10);
+    }
+    
+    @Test
+    public void testPostContructionWhenRestoreDoesNotExist() throws Exception {
+        
+        byte[] primaryBytes = "<users/>".getBytes();
+        FileOutputStream fos = new FileOutputStream(primary);
+        fos.write(primaryBytes);
+        fos.close();
+        
+        provider.onConfigured(mockConfigurationContext);
+        assertEquals(primary.length(), restore.length());
+    }
+    
+    @Test
+    public void testPostContructionWhenPrimaryDoesNotExist() throws Exception {
+        
+        byte[] restoreBytes = "<users/>".getBytes();
+        FileOutputStream fos = new FileOutputStream(restore);
+        fos.write(restoreBytes);
+        fos.close();
+        
+        provider.onConfigured(mockConfigurationContext);
+        assertEquals(restore.length(), primary.length());
+        
+    }
+    
+    @Test(expected = ProviderCreationException.class)
+    public void testPostContructionWhenPrimaryDifferentThanRestore() throws Exception {
+        
+        byte[] primaryBytes = "<users></users>".getBytes();
+        FileOutputStream fos = new FileOutputStream(primary);
+        fos.write(primaryBytes);
+        fos.close();
+        
+        byte[] restoreBytes = "<users/>".getBytes();
+        fos = new FileOutputStream(restore);
+        fos.write(restoreBytes);
+        fos.close();
+        
+        provider.onConfigured(mockConfigurationContext);
+    }
+    
+    @Test
+    public void testPostContructionWhenPrimaryAndBackupDoNotExist() throws Exception {
+        
+        provider.onConfigured(mockConfigurationContext);
+        assertEquals(0, restore.length());
+        assertEquals(restore.length(), primary.length());
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/misc/build-order.sh
----------------------------------------------------------------------
diff --git a/misc/build-order.sh b/misc/build-order.sh
new file mode 100755
index 0000000..855321a
--- /dev/null
+++ b/misc/build-order.sh
@@ -0,0 +1,79 @@
+#MAVEN_FLAGS="-Dmaven.test.skip=true"
+MAVEN_FLAGS=""
+
+cd misc/nar-maven-plugin && \
+mvn $MAVEN_FLAGS install && \
+cd ../../commons/nifi-parent && \
+mvn $MAVEN_FLAGS install && \
+cd ../../nifi-api && \
+mvn $MAVEN_FLAGS install && \
+cd ../commons/ && \
+cd	nifi-stream-utils && \
+mvn $MAVEN_FLAGS install && \
+cd	../wali && \
+mvn $MAVEN_FLAGS install && \
+cd	../flowfile-packager && \
+mvn $MAVEN_FLAGS install && \
+cd	../core-flowfile-attributes && \
+mvn $MAVEN_FLAGS install && \
+cd	../data-provenance-utils && \
+mvn $MAVEN_FLAGS install && \
+cd	../naive-search-ring-buffer && \
+mvn $MAVEN_FLAGS install && \
+cd	../nifi-expression-language && \
+mvn $MAVEN_FLAGS install && \
+cd	../nifi-file-utils && \
+mvn $MAVEN_FLAGS install && \
+cd	../nifi-logging-utils && \
+mvn $MAVEN_FLAGS install && \
+cd	../nifi-properties && \
+mvn $MAVEN_FLAGS install && \
+cd	../nifi-security-utils && \
+mvn $MAVEN_FLAGS install && \
+cd	../nifi-utils && \
+mvn $MAVEN_FLAGS install && \
+cd	../nifi-socket-utils && \
+mvn $MAVEN_FLAGS install && \
+cd	../nifi-web-utils && \
+mvn $MAVEN_FLAGS install && \
+cd	../processor-utilities && \
+mvn $MAVEN_FLAGS install && \
+cd	../remote-communications-utils && \
+mvn $MAVEN_FLAGS install && \
+cd	../search-utils && \
+mvn $MAVEN_FLAGS install && \
+cd ../../extensions/file-authorization-provider && \
+mvn $MAVEN_FLAGS install && \
+cd ../../nifi-mock && \
+mvn $MAVEN_FLAGS install && \
+cd ../nar-bundles/ && \
+cd	nar-container-common && \
+mvn $MAVEN_FLAGS install && \
+cd	../jetty-bundle && \
+mvn $MAVEN_FLAGS install && \
+cd	../standard-services-api-bundle && \
+mvn $MAVEN_FLAGS install && \
+cd	../ssl-context-bundle && \
+mvn $MAVEN_FLAGS install && \
+cd	../distributed-cache-services-bundle && \
+mvn $MAVEN_FLAGS install && \
+cd	../standard-bundle && \
+mvn $MAVEN_FLAGS install && \
+cd	../hadoop-libraries-bundle && \
+mvn $MAVEN_FLAGS install && \
+cd	../hadoop-bundle && \
+mvn $MAVEN_FLAGS install && \
+cd	../volatile-provenance-repository-bundle && \
+mvn $MAVEN_FLAGS install && \
+cd	../persistent-provenance-repository-bundle && \
+mvn $MAVEN_FLAGS install && \
+cd	../framework-bundle && \
+mvn $MAVEN_FLAGS install && \
+cd	../execute-script-bundle && \
+mvn $MAVEN_FLAGS install && \
+cd	../monitor-threshold-bundle && \
+mvn $MAVEN_FLAGS install && \
+cd	../update-attribute-bundle && \
+mvn $MAVEN_FLAGS install && \
+cd ../../assemblies/nifi 
+mvn assembly:assembly

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/misc/nar-maven-plugin/pom.xml
----------------------------------------------------------------------
diff --git a/misc/nar-maven-plugin/pom.xml b/misc/nar-maven-plugin/pom.xml
new file mode 100644
index 0000000..3888df3
--- /dev/null
+++ b/misc/nar-maven-plugin/pom.xml
@@ -0,0 +1,83 @@
+<?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>
+    <groupId>org.apache.nifi</groupId>
+    <artifactId>nar-maven-plugin</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <packaging>maven-plugin</packaging>
+    <name>Apache NiFi NAR Plugin</name>
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    </properties>
+    <build>
+        <defaultGoal>install</defaultGoal>
+        <plugins>
+            <plugin>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <version>3.2</version>
+                <configuration>
+                    <source>1.7</source>
+                    <target>1.7</target>
+                </configuration>
+            </plugin>
+            <plugin>
+                <artifactId>maven-jar-plugin</artifactId>
+                <version>2.5</version>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-plugin-plugin</artifactId>
+                <version>3.3</version>
+            </plugin>
+        </plugins>
+    </build>    
+    <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>
+            <!-- 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>
+            <version>2.5</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.maven.plugin-tools</groupId>
+            <artifactId>maven-plugin-annotations</artifactId>
+            <version>3.3</version>
+        </dependency>        
+    </dependencies>
+    <distributionManagement>
+        <repository>
+            <id>nifi-releases</id>
+            <url>${nifi.repo.url}</url>
+        </repository>
+        <snapshotRepository>
+            <id>nifi-snapshots</id>
+            <url>${nifi.snapshot.repo.url}</url>
+        </snapshotRepository>
+    </distributionManagement>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/misc/nar-maven-plugin/src/main/java/nifi/NarMojo.java
----------------------------------------------------------------------
diff --git a/misc/nar-maven-plugin/src/main/java/nifi/NarMojo.java b/misc/nar-maven-plugin/src/main/java/nifi/NarMojo.java
new file mode 100644
index 0000000..263fe88
--- /dev/null
+++ b/misc/nar-maven-plugin/src/main/java/nifi/NarMojo.java
@@ -0,0 +1,610 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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(property = "project", readonly = true, required = true)
+    protected MavenProject project;
+
+    @Parameter(property = "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;
+
+    /**
+     * 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 = new File(project.getBasedir(), "target");
+        return new File(outputDirectory, "classes");
+    }
+
+    private File getDependenciesDirectory() {
+        return new File(getClassesDirectory(), "META-INF/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 = new File(project.getBasedir(), "target");
+        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/4d998c12/misc/nar-maven-plugin/src/main/resources/META-INF/plexus/components.xml
----------------------------------------------------------------------
diff --git a/misc/nar-maven-plugin/src/main/resources/META-INF/plexus/components.xml b/misc/nar-maven-plugin/src/main/resources/META-INF/plexus/components.xml
new file mode 100644
index 0000000..0680d18
--- /dev/null
+++ b/misc/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/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/pom.xml b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/pom.xml
new file mode 100644
index 0000000..6280349
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/pom.xml
@@ -0,0 +1,67 @@
+<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>distributed-cache-services-bundle</artifactId>
+		<version>0.0.1-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>distributed-cache-client-service</artifactId>
+	<packaging>jar</packaging>
+
+	<name>Distributed Cache Client Service</name>
+	<description>Provides a Client for interfacing with a Distributed Cache</description>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>nifi-api</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>distributed-cache-client-service-api</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>distributed-cache-protocol</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>remote-communications-utils</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>nifi-processor-utils</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>nifi-stream-utils</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>ssl-context-service-api</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>junit</groupId>
+			<artifactId>junit</artifactId>
+			<version>4.9</version>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/CommsSession.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/CommsSession.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/CommsSession.java
new file mode 100644
index 0000000..f838c2f
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/CommsSession.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.TimeUnit;
+
+import javax.net.ssl.SSLContext;
+
+public interface CommsSession extends Closeable {
+
+    void setTimeout(final long value, final TimeUnit timeUnit);
+    
+    InputStream getInputStream() throws IOException;
+    
+    OutputStream getOutputStream() throws IOException;
+    
+    boolean isClosed();
+    
+    void interrupt();
+    
+    String getHostname();
+    
+    int getPort();
+    
+    long getTimeout(TimeUnit timeUnit);
+    
+    SSLContext getSSLContext();
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java
new file mode 100644
index 0000000..ee96660
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java
@@ -0,0 +1,301 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.client;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.annotation.OnConfigured;
+import org.apache.nifi.distributed.cache.protocol.ProtocolHandshake;
+import org.apache.nifi.distributed.cache.protocol.exception.HandshakeException;
+import org.apache.nifi.io.ByteArrayOutputStream;
+import org.apache.nifi.io.DataOutputStream;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.remote.StandardVersionNegotiator;
+import org.apache.nifi.remote.VersionNegotiator;
+import org.apache.nifi.ssl.SSLContextService;
+import org.apache.nifi.ssl.SSLContextService.ClientAuth;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DistributedMapCacheClientService extends AbstractControllerService implements DistributedMapCacheClient {
+
+    private static final Logger logger = LoggerFactory.getLogger(DistributedMapCacheClientService.class);
+
+    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor.Builder()
+            .name("Server Hostname")
+            .description("The name of the server that is running the DistributedMapCacheServer service")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+            .name("Server Port")
+            .description("The port on the remote server that is to be used when communicating with the DistributedMapCacheServer service")
+            .required(true)
+            .addValidator(StandardValidators.PORT_VALIDATOR)
+            .defaultValue("4557")
+            .build();
+    public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
+            .name("SSL Context Service")
+            .description(
+                    "If specified, indicates the SSL Context Service that is used to communicate with the remote server. If not specified, communications will not be encrypted")
+            .required(false)
+            .addValidator(StandardValidators.createControllerServiceExistsValidator(SSLContextService.class))
+            .defaultValue(null)
+            .build();
+    public static final PropertyDescriptor COMMUNICATIONS_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("Communications Timeout")
+            .description(
+                    "Specifies how long to wait when communicating with the remote server before determining that there is a communications failure if data cannot be sent or received")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .build();
+
+    private final BlockingQueue<CommsSession> queue = new LinkedBlockingQueue<>();
+    private volatile ConfigurationContext configContext;
+    private volatile boolean closed = false;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(HOSTNAME);
+        descriptors.add(PORT);
+        descriptors.add(SSL_CONTEXT_SERVICE);
+        descriptors.add(COMMUNICATIONS_TIMEOUT);
+        return descriptors;
+    }
+
+    @OnConfigured
+    public void cacheConfig(final ConfigurationContext context) {
+        this.configContext = context;
+    }
+
+    @Override
+    public <K, V> boolean putIfAbsent(final K key, final V value, final Serializer<K> keySerializer, final Serializer<V> valueSerializer)
+            throws IOException {
+        return withCommsSession(new CommsAction<Boolean>() {
+            @Override
+            public Boolean execute(final CommsSession session) throws IOException {
+                final DataOutputStream dos = new DataOutputStream(session.getOutputStream());
+                dos.writeUTF("putIfAbsent");
+
+                serialize(key, keySerializer, dos);
+                serialize(value, valueSerializer, dos);
+
+                dos.flush();
+
+                final DataInputStream dis = new DataInputStream(session.getInputStream());
+                return dis.readBoolean();
+            }
+        });
+    }
+
+    @Override
+    public <K> boolean containsKey(final K key, final Serializer<K> keySerializer) throws IOException {
+        return withCommsSession(new CommsAction<Boolean>() {
+            @Override
+            public Boolean execute(final CommsSession session) throws IOException {
+                final DataOutputStream dos = new DataOutputStream(session.getOutputStream());
+                dos.writeUTF("containsKey");
+
+                serialize(key, keySerializer, dos);
+                dos.flush();
+
+                final DataInputStream dis = new DataInputStream(session.getInputStream());
+                return dis.readBoolean();
+            }
+        });
+    }
+
+    @Override
+    public <K, V> V getAndPutIfAbsent(final K key, final V value, final Serializer<K> keySerializer, final Serializer<V> valueSerializer,
+            final Deserializer<V> valueDeserializer) throws IOException {
+        return withCommsSession(new CommsAction<V>() {
+            @Override
+            public V execute(final CommsSession session) throws IOException {
+                final DataOutputStream dos = new DataOutputStream(session.getOutputStream());
+                dos.writeUTF("getAndPutIfAbsent");
+
+                serialize(key, keySerializer, dos);
+                serialize(value, valueSerializer, dos);
+                dos.flush();
+
+                // read response
+                final DataInputStream dis = new DataInputStream(session.getInputStream());
+                final byte[] responseBuffer = readLengthDelimitedResponse(dis);
+                return valueDeserializer.deserialize(responseBuffer);
+            }
+        });
+    }
+
+    @Override
+    public <K, V> V get(final K key, final Serializer<K> keySerializer, final Deserializer<V> valueDeserializer) throws IOException {
+        return withCommsSession(new CommsAction<V>() {
+            @Override
+            public V execute(final CommsSession session) throws IOException {
+                final DataOutputStream dos = new DataOutputStream(session.getOutputStream());
+                dos.writeUTF("get");
+
+                serialize(key, keySerializer, dos);
+                dos.flush();
+
+                // read response
+                final DataInputStream dis = new DataInputStream(session.getInputStream());
+                final byte[] responseBuffer = readLengthDelimitedResponse(dis);
+                return valueDeserializer.deserialize(responseBuffer);
+            }
+        });
+    }
+
+    @Override
+    public <K> boolean remove(final K key, final Serializer<K> serializer) throws IOException {
+        return withCommsSession(new CommsAction<Boolean>() {
+            @Override
+            public Boolean execute(final CommsSession session) throws IOException {
+                final DataOutputStream dos = new DataOutputStream(session.getOutputStream());
+                dos.writeUTF("remove");
+
+                serialize(key, serializer, dos);
+                dos.flush();
+
+                // read response
+                final DataInputStream dis = new DataInputStream(session.getInputStream());
+                return dis.readBoolean();
+            }
+        });
+    }
+
+    private byte[] readLengthDelimitedResponse(final DataInputStream dis) throws IOException {
+        final int responseLength = dis.readInt();
+        final byte[] responseBuffer = new byte[responseLength];
+        dis.readFully(responseBuffer);
+        return responseBuffer;
+    }
+
+    public CommsSession createCommsSession(final ConfigurationContext context) throws IOException {
+        final String hostname = context.getProperty(HOSTNAME).getValue();
+        final int port = context.getProperty(PORT).asInteger();
+        final long timeoutMillis = context.getProperty(COMMUNICATIONS_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS);
+        final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+
+        final CommsSession commsSession;
+        if (sslContextService == null) {
+            commsSession = new StandardCommsSession(hostname, port);
+        } else {
+            commsSession = new SSLCommsSession(sslContextService.createSSLContext(ClientAuth.REQUIRED), hostname, port);
+        }
+
+        commsSession.setTimeout(timeoutMillis, TimeUnit.MILLISECONDS);
+        return commsSession;
+    }
+
+    private CommsSession leaseCommsSession() throws IOException {
+        CommsSession session = queue.poll();
+        if (session != null && !session.isClosed()) {
+            return session;
+        }
+
+        session = createCommsSession(configContext);
+        final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(1);
+        try {
+            ProtocolHandshake.initiateHandshake(session.getInputStream(), session.getOutputStream(), versionNegotiator);
+        } catch (final HandshakeException e) {
+            try {
+                session.close();
+            } catch (final IOException ioe) {
+            }
+
+            throw new IOException(e);
+        }
+
+        return session;
+    }
+
+    @Override
+    public void close() throws IOException {
+        this.closed = true;
+
+        CommsSession commsSession;
+        while ((commsSession = queue.poll()) != null) {
+            try (final DataOutputStream dos = new DataOutputStream(commsSession.getOutputStream())) {
+                dos.writeUTF("close");
+                dos.flush();
+                commsSession.close();
+            } catch (final IOException e) {
+            }
+        }
+        logger.info("Closed {}", new Object[] { getIdentifier() });
+    }
+
+    @Override
+    protected void finalize() throws Throwable {
+        if (!closed)
+            close();
+        logger.debug("Finalize called");
+    }
+
+    private <T> void serialize(final T value, final Serializer<T> serializer, final DataOutputStream dos) throws IOException {
+        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        serializer.serialize(value, baos);
+        dos.writeInt(baos.size());
+        baos.writeTo(dos);
+    }
+
+    private <T> T withCommsSession(final CommsAction<T> action) throws IOException {
+        if (closed) {
+            throw new IllegalStateException("Client is closed");
+        }
+
+        final CommsSession session = leaseCommsSession();
+        try {
+            return action.execute(session);
+        } catch (final IOException ioe) {
+            try {
+                session.close();
+            } catch (final IOException ignored) {
+            }
+
+            throw ioe;
+        } finally {
+            if (!session.isClosed()) {
+                if (this.closed) {
+                    try {
+                        session.close();
+                    } catch (final IOException ioe) {
+                    }
+                } else {
+                    queue.offer(session);
+                }
+            }
+        }
+    }
+
+    private static interface CommsAction<T> {
+        T execute(CommsSession commsSession) throws IOException;
+    }
+
+}


[28/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..76e54d6
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/UserService.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Set;
+import org.apache.nifi.authorization.Authority;
+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();
+
+    /**
+     * 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/4d998c12/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
new file mode 100644
index 0000000..41c97fe
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AbstractUserAction.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..5a2159f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AddActionsAction.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..5818ebe
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AdministrationAction.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ea6973d
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeUserAction.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..3833abb
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/CreateUserAction.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f93e97e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DeleteUserAction.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..c31f107
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserAction.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..385fce6
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserGroupAction.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..8e5b574
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/FindUserByDnAction.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..3062a2e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/FindUserByIdAction.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..1dc5588
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetActionAction.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..3b82d79
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetActionsAction.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..5ce663e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..5377c46
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetUserGroupAction.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..42d180e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetUsersAction.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..3325642
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/HasPendingUserAccounts.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..14596b2
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/InvalidateUserAccountAction.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..0cb7e14
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/InvalidateUserGroupAccountsAction.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b5a2883
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/PurgeActionsAction.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..3dce6d9
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/RequestUserAccountAction.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..72d68db
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/SeedUserAccountsAction.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..01eaf5f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserAction.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..fa24fbe
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserGroupAction.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..cef21d7
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAction.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+}


[19/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..a6542c8
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserGroupEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..baffe15
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserSearchResultsEntity.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..2d11d1f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UsersEntity.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100755
index 0000000..ea8c4bf
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/.gitignore
@@ -0,0 +1 @@
+/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..b8960c3
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/pom.xml
@@ -0,0 +1,48 @@
+<?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/4d998c12/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
new file mode 100644
index 0000000..2b3b38c
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/ClusterManagerAuthorizationProvider.java
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..c81e9d0
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/NodeAuthorizationProvider.java
@@ -0,0 +1,381 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.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 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/4d998c12/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
new file mode 100644
index 0000000..38d0dd8
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/DoesDnExistMessage.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..347163f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetAuthoritiesMessage.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..717f244
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetGroupForUserMessage.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..102142a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/ProtocolMessage.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..97a1bc7
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/JaxbProtocolUtils.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..5cde335
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/ObjectFactory.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..56f4c3e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100755
index 0000000..ea8c4bf
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/.gitignore
@@ -0,0 +1 @@
+/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..5351085
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/pom.xml
@@ -0,0 +1,69 @@
+<?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/4d998c12/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
new file mode 100644
index 0000000..fa1547f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1b5d007
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionRequest.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..7a5ff2b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..67324a1
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..a120524
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeBulletins.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..1893186
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 + ']';
+    }
+
+}


[30/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..f5d3ecc
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.js
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..91e6ca7
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.py
@@ -0,0 +1,32 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..c10765d
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.rb
@@ -0,0 +1,30 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..e34ad5f
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.js
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..23e55af
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.py
@@ -0,0 +1,32 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..308b652
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.rb
@@ -0,0 +1,33 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..47ef546
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.js
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+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;
+        }
+    });
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..ed5408a
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.py
@@ -0,0 +1,37 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..38249ae
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.rb
@@ -0,0 +1,39 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..72bb80e
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.js
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..d52368e
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.py
@@ -0,0 +1,60 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..f333294
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.rb
@@ -0,0 +1,42 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+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/4d998c12/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
new file mode 100644
index 0000000..1e055b7
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.js
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..c563b66
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.py
@@ -0,0 +1,22 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..5be4553
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.rb
@@ -0,0 +1,32 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..1cc28dc
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/nar/pom.xml
@@ -0,0 +1,36 @@
+<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/4d998c12/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
new file mode 100644
index 0000000..b3abb36
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/pom.xml
@@ -0,0 +1,81 @@
+<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/4d998c12/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
new file mode 100755
index 0000000..ea8c4bf
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/.gitignore
@@ -0,0 +1 @@
+/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..b5dd171
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/pom.xml
@@ -0,0 +1,116 @@
+<?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/4d998c12/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
new file mode 100644
index 0000000..aeb2755
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/AuditDataSourceFactoryBean.java
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b95388b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/RepositoryUtils.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1f64f6e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/UserDataSourceFactoryBean.java
@@ -0,0 +1,247 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..5d6d222
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..2992884
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/AuthorityDAO.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..dee4ef9
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/DAOFactory.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..05bf4af
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/DataAccessException.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..9ffab5d
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/UserDAO.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..2f3de0e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/DAOFactoryImpl.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+    }
+
+}


[07/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..e434905
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/RepositoryStatusReport.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..6f9c237
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..35a255d
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..5cb676f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..d1d5e5b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..9846cf2
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/BulletinProcessingStrategy.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..23c4cdb
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/ComponentBulletin.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f97dc46
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/SystemBulletin.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..61be59c
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
@@ -0,0 +1,723 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..3eb594b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/ProcessGroupCounts.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..3acd1d3
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+
+    /**
+     * 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/4d998c12/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
new file mode 100644
index 0000000..fb4f6e0
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroupPortDescriptor.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..27cc6c5
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogMessage.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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);
+        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/4d998c12/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
new file mode 100644
index 0000000..a75f8ea
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogObserver.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..4a017ce
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogRepository.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..76ca661
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/logging/LogRepositoryFactory.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b25c90b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/nar/NarCloseable.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..aa905a8
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..2422fe1
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/Peer.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..d1cb076
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PeerStatus.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..8f2603a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PortAuthorizationResult.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..12a3d33
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteAuthorizationState.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote;
+
+/**
+ *
+ */
+public enum RemoteAuthorizationState {
+
+    UNKNOWN,
+    UNAUTHORIZED,
+    AUTHORIZED;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..d4ad374
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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();
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..4afdfb7
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RootGroupPort.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+}


[06/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..56432d5
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/TransferDirection.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote;
+
+public enum TransferDirection {
+
+    SEND,
+    RECEIVE;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..bfccd98
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote;
+
+public interface VersionedRemoteResource {
+
+    VersionNegotiator getVersionNegotiator();
+
+    String getResourceName();
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..b4206b3
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f6c2f4f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/BadRequestException.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b61fc65
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..24ff3a5
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/NotAuthorizedException.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..af0f467
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..0f50b98
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..dd675b3
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/RequestExpiredException.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..e6a0fe7
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..32274eb
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..d2e2946
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..95cab29
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..d009cec
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..41334fe
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/RequestType.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..0d18f2e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100755
index 0000000..ea8c4bf
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/.gitignore
@@ -0,0 +1 @@
+/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..547c75d
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/pom.xml
@@ -0,0 +1,130 @@
+<?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-file-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>org.apache.nifi</groupId>
+            <artifactId>nifi-core-flowfile-attributes</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>wali</groupId>
+            <artifactId>wali</artifactId>
+            <version>3.0.0-SNAPSHOT</version>
+        </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/4d998c12/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
new file mode 100644
index 0000000..1249657
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/BulletinsPayload.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..986e904
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/ConnectionException.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..55707f3
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/DisconnectionException.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..093b238
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/cluster/HeartbeatPayload.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1efa0cd
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/connectable/LocalPort.java
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.EVENT_DRIVEN;
+    }
+
+    @Override
+    public boolean isSideEffectFree() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..1d723b5
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/connectable/StandardConnection.java
@@ -0,0 +1,336 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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");
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..f36a459
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/EventDrivenWorkerQueue.java
@@ -0,0 +1,329 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+}


[17/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..dc86d24
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..4b359f4
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ca30d9b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..bc68630
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/JaxbProtocolContext.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..d9de24f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionRequest.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..c7c783b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..89d903b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedCounter.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..bb97619
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedDataFlow.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..5b9d9b7
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..98e2438
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeBulletins.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..8134ea3
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeIdentifier.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1f91cf1
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionRequestAdapter.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..143bab0
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..8d9467f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/DataFlowAdapter.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..0e073b6
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..c3a57f5
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..1ae41f7
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeBulletinsAdapter.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..fe2d8a4
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeIdentifierAdapter.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1613536
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..344de4e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionRequestMessage.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..a262d7a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionResponseMessage.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ebc1cae
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ControllerStartupFailureMessage.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..8aa7a40
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/DisconnectMessage.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..99a6dee
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ExceptionMessage.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..4a10538
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowRequestMessage.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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;
+    }
+    
+}


[41/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..fc279fb
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/ServerSocketConfiguration.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..c24b540
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketConfiguration.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..e02791a
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketListener.java
@@ -0,0 +1,211 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..fb6a00c
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketUtils.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..7a62813
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/DiscoverableService.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..5f378b9
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/DiscoverableServiceImpl.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ea0b72a
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastConfiguration.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..e562c25
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastListener.java
@@ -0,0 +1,193 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..c254c11
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastServiceDiscovery.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..a3cff9b
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastServicesBroadcaster.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..dad1173
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastTimeToLive.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..8a8b7c0
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastUtils.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..173146e
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/ServiceDiscovery.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..86260d8
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/ServicesBroadcaster.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b5240c9
--- /dev/null
+++ b/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/ServerMain.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b3d214e
--- /dev/null
+++ b/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/TCPClient.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..90f4c42
--- /dev/null
+++ b/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/UDPClient.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..9ec26e9
--- /dev/null
+++ b/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/UselessStreamConsumer.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..8e93769
--- /dev/null
+++ b/commons/nifi-socket-utils/src/test/resources/log4j.xml
@@ -0,0 +1,36 @@
+<?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/4d998c12/commons/nifi-stream-utils/.gitignore
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/.gitignore b/commons/nifi-stream-utils/.gitignore
new file mode 100755
index 0000000..ea8c4bf
--- /dev/null
+++ b/commons/nifi-stream-utils/.gitignore
@@ -0,0 +1 @@
+/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/nifi-stream-utils/pom.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/pom.xml b/commons/nifi-stream-utils/pom.xml
new file mode 100644
index 0000000..0413575
--- /dev/null
+++ b/commons/nifi-stream-utils/pom.xml
@@ -0,0 +1,42 @@
+<?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-parent</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-stream-utils</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <packaging>jar</packaging>
+    <name>NiFi Stream Utils</name>
+    
+    <build>
+        <plugins>
+            <!-- Enforce 1.6 compliance -->
+            <plugin>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <configuration>
+                    <source>1.6</source>
+                    <target>1.6</target>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+    
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/BufferedInputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/BufferedInputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/BufferedInputStream.java
new file mode 100644
index 0000000..57adb8c
--- /dev/null
+++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/BufferedInputStream.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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/4d998c12/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/BufferedOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/BufferedOutputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/BufferedOutputStream.java
new file mode 100644
index 0000000..56caf65
--- /dev/null
+++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/BufferedOutputStream.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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();
+    }
+}


[39/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/nifi-stream-utils/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/test/resources/logback-test.xml b/commons/nifi-stream-utils/src/test/resources/logback-test.xml
new file mode 100644
index 0000000..0f3f60c
--- /dev/null
+++ b/commons/nifi-stream-utils/src/test/resources/logback-test.xml
@@ -0,0 +1,48 @@
+<?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"/>
+    
+    <!-- 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/4d998c12/commons/nifi-utils/.gitignore
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/.gitignore b/commons/nifi-utils/.gitignore
new file mode 100755
index 0000000..12c5231
--- /dev/null
+++ b/commons/nifi-utils/.gitignore
@@ -0,0 +1,8 @@
+/target
+/target
+/target
+/target
+/target
+/target
+/target
+/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/nifi-utils/pom.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/pom.xml b/commons/nifi-utils/pom.xml
new file mode 100644
index 0000000..8aeccd7
--- /dev/null
+++ b/commons/nifi-utils/pom.xml
@@ -0,0 +1,28 @@
+<?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-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>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/nifi-utils/src/main/java/org/apache/nifi/io/CompoundUpdateMonitor.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/io/CompoundUpdateMonitor.java b/commons/nifi-utils/src/main/java/org/apache/nifi/io/CompoundUpdateMonitor.java
new file mode 100644
index 0000000..e22032b
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/io/CompoundUpdateMonitor.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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/4d998c12/commons/nifi-utils/src/main/java/org/apache/nifi/io/LastModifiedMonitor.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/io/LastModifiedMonitor.java b/commons/nifi-utils/src/main/java/org/apache/nifi/io/LastModifiedMonitor.java
new file mode 100644
index 0000000..f446465
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/io/LastModifiedMonitor.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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/4d998c12/commons/nifi-utils/src/main/java/org/apache/nifi/io/MD5SumMonitor.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/io/MD5SumMonitor.java b/commons/nifi-utils/src/main/java/org/apache/nifi/io/MD5SumMonitor.java
new file mode 100644
index 0000000..1326c2a
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/io/MD5SumMonitor.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.io;
+
+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)) > 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/4d998c12/commons/nifi-utils/src/main/java/org/apache/nifi/io/SynchronousFileWatcher.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/io/SynchronousFileWatcher.java b/commons/nifi-utils/src/main/java/org/apache/nifi/io/SynchronousFileWatcher.java
new file mode 100644
index 0000000..785f1ac
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/io/SynchronousFileWatcher.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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/4d998c12/commons/nifi-utils/src/main/java/org/apache/nifi/io/UpdateMonitor.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/io/UpdateMonitor.java b/commons/nifi-utils/src/main/java/org/apache/nifi/io/UpdateMonitor.java
new file mode 100644
index 0000000..33fb444
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/io/UpdateMonitor.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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/4d998c12/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
new file mode 100644
index 0000000..92061e0
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..9954bfb
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/FormatUtils.java
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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");
+        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").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/4d998c12/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
new file mode 100644
index 0000000..213bbc0
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ef70ce8
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..a58ec6a
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/ObjectHolder.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..c0bb830
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/RingBuffer.java
@@ -0,0 +1,292 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..cd11930
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/StopWatch.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..63736ed
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/Tuple.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..a8d7e82
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebugDisabledTimedLock.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f082168
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebugEnabledTimedLock.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..69da6e8
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebuggableTimedLock.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..532d3c3
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/TimedLock.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..2b95897
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/EntityAccess.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..193abc6
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/LongEntityAccess.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..dd8e523
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/TimedBuffer.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..07d31ea
--- /dev/null
+++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/TimestampedLong.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestCompoundUpdateMonitor.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestCompoundUpdateMonitor.java b/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestCompoundUpdateMonitor.java
new file mode 100644
index 0000000..c796a96
--- /dev/null
+++ b/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestCompoundUpdateMonitor.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.apache.nifi.io.CompoundUpdateMonitor;
+import org.apache.nifi.io.LastModifiedMonitor;
+import org.apache.nifi.io.MD5SumMonitor;
+import org.apache.nifi.io.UpdateMonitor;
+
+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/4d998c12/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
new file mode 100644
index 0000000..fafffdd
--- /dev/null
+++ b/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestRingBuffer.java
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestSynchronousFileWatcher.java
----------------------------------------------------------------------
diff --git a/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestSynchronousFileWatcher.java b/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestSynchronousFileWatcher.java
new file mode 100644
index 0000000..4b2c0d5
--- /dev/null
+++ b/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestSynchronousFileWatcher.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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;
+
+import org.apache.nifi.io.MD5SumMonitor;
+import org.apache.nifi.io.SynchronousFileWatcher;
+import org.apache.nifi.io.UpdateMonitor;
+
+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());
+    }
+}


[26/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..8536871
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/History.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..53cc13c
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/HistoryQuery.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..6ece5cf
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/PreviousValue.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..195f6bf
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/AccountStatus.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..984a572
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUser.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+    /* 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 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/4d998c12/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
new file mode 100644
index 0000000..7586fd1
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUserGroup.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..a36619f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/resources/nifi-administration-context.xml
@@ -0,0 +1,62 @@
+<?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/4d998c12/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
new file mode 100644
index 0000000..122fa2c
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/xsd/authority-providers.xsd
@@ -0,0 +1,49 @@
+<?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/4d998c12/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
new file mode 100644
index 0000000..5797705
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/AuthorizeUserActionTest.java
@@ -0,0 +1,433 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..3d2081b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/CreateUserActionTest.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..de85298
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/DisableUserActionTest.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..93dbe61
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/InvalidateUserAccountActionTest.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..6e77d46
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/RequestUserAccountActionTest.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f37fc84
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/SeedUserAccountsActionTest.java
@@ -0,0 +1,263 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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));
+    }
+}


[13/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..412a555
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/DisconnectedNodeMutableRequestException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..6c4e670
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalClusterStateException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..adef62a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDeletionException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..7e61b24
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDisconnectionException.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..96c76bc
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeReconnectionException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..4b0097a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IneligiblePrimaryNodeException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..d160587
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/MutableRequestException.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..8d704b9
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoConnectedNodesException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..9e17a23
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoResponseFromNodesException.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..3bd2f4b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeDisconnectionException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..8c40cef
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeReconnectionException.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..403f7a5
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/PrimaryRoleAssignmentException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..f544f26
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/SafeModeMutableRequestException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..914bb56
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UnknownNodeException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..773d7b5
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UriConstructionException.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..2015530
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..e546f87
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredReportingContext.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..81bb7a7
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java
@@ -0,0 +1,531 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..afade7e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+
+}


[08/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..3b880bb
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java
@@ -0,0 +1,1093 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+                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/4d998c12/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
new file mode 100644
index 0000000..52a4e40
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
@@ -0,0 +1,541 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.EVENT_DRIVEN;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..df3c251
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..2f43600
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/WorkerQueue.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..368ed1b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/CommunicationsException.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..0ff68b0
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceAlreadyExistsException.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..4cdbe54
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceNotFoundException.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..c4aba44
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorInstantiationException.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..5acca16
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorLifeCycleException.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..97c44b5
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/label/Label.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ced6ff9
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskInstantiationException.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..6ce7ba6
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/ContentNotFoundException.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..de231ed
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/CounterRepository.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f07a530
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/FlowFileEvent.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..2eb3caf
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/FlowFileEventRepository.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+}


[40/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteArrayInputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteArrayInputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteArrayInputStream.java
new file mode 100644
index 0000000..9defab4
--- /dev/null
+++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteArrayInputStream.java
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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/4d998c12/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteArrayOutputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteArrayOutputStream.java
new file mode 100644
index 0000000..b504c23
--- /dev/null
+++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteArrayOutputStream.java
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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/4d998c12/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteCountingInputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteCountingInputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteCountingInputStream.java
new file mode 100644
index 0000000..792cc32
--- /dev/null
+++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteCountingInputStream.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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/4d998c12/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteCountingOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteCountingOutputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteCountingOutputStream.java
new file mode 100644
index 0000000..c7b77ff
--- /dev/null
+++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteCountingOutputStream.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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/4d998c12/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/DataOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/DataOutputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/DataOutputStream.java
new file mode 100644
index 0000000..6af06d3
--- /dev/null
+++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/DataOutputStream.java
@@ -0,0 +1,417 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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/4d998c12/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/GZIPOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/GZIPOutputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/GZIPOutputStream.java
new file mode 100644
index 0000000..875b838
--- /dev/null
+++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/GZIPOutputStream.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/LeakyBucketStreamThrottler.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/LeakyBucketStreamThrottler.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/LeakyBucketStreamThrottler.java
new file mode 100644
index 0000000..0ebe16d
--- /dev/null
+++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/LeakyBucketStreamThrottler.java
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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 {
+                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/4d998c12/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NonCloseableInputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NonCloseableInputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NonCloseableInputStream.java
new file mode 100644
index 0000000..1fbb093
--- /dev/null
+++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NonCloseableInputStream.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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/4d998c12/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NonCloseableOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NonCloseableOutputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NonCloseableOutputStream.java
new file mode 100644
index 0000000..731e409
--- /dev/null
+++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NonCloseableOutputStream.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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/4d998c12/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NullOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NullOutputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NullOutputStream.java
new file mode 100644
index 0000000..60475d4
--- /dev/null
+++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NullOutputStream.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/StreamThrottler.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/StreamThrottler.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/StreamThrottler.java
new file mode 100644
index 0000000..8c2aa80
--- /dev/null
+++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/StreamThrottler.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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/4d998c12/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/StreamUtils.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/StreamUtils.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/StreamUtils.java
new file mode 100644
index 0000000..1596014
--- /dev/null
+++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/StreamUtils.java
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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.io.exception.BytePatternNotFoundException;
+import org.apache.nifi.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/4d998c12/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ZipOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ZipOutputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ZipOutputStream.java
new file mode 100644
index 0000000..f285720
--- /dev/null
+++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ZipOutputStream.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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/4d998c12/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/exception/BytePatternNotFoundException.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/exception/BytePatternNotFoundException.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/exception/BytePatternNotFoundException.java
new file mode 100644
index 0000000..8935767
--- /dev/null
+++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/exception/BytePatternNotFoundException.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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/4d998c12/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/util/NonThreadSafeCircularBuffer.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/util/NonThreadSafeCircularBuffer.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/util/NonThreadSafeCircularBuffer.java
new file mode 100644
index 0000000..1b87488
--- /dev/null
+++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/util/NonThreadSafeCircularBuffer.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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/4d998c12/commons/nifi-stream-utils/src/test/java/org/apache/nifi/io/TestLeakyBucketThrottler.java
----------------------------------------------------------------------
diff --git a/commons/nifi-stream-utils/src/test/java/org/apache/nifi/io/TestLeakyBucketThrottler.java b/commons/nifi-stream-utils/src/test/java/org/apache/nifi/io/TestLeakyBucketThrottler.java
new file mode 100644
index 0000000..12e1801
--- /dev/null
+++ b/commons/nifi-stream-utils/src/test/java/org/apache/nifi/io/TestLeakyBucketThrottler.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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");
+        }
+    }
+
+}


[38/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..39ca330
--- /dev/null
+++ b/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestTimedBuffer.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/nifi-web-utils/pom.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-web-utils/pom.xml b/commons/nifi-web-utils/pom.xml
new file mode 100644
index 0000000..434e1a3
--- /dev/null
+++ b/commons/nifi-web-utils/pom.xml
@@ -0,0 +1,56 @@
+<?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-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>
+            <version>0.0.1-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-codec</groupId>
+            <artifactId>commons-codec</artifactId>
+            <version>1.10</version>
+        </dependency>
+        <dependency>
+            <groupId>com.sun.jersey</groupId>
+            <artifactId>jersey-client</artifactId>
+            <version>1.18.2</version>
+        </dependency>
+        <dependency>
+            <groupId>com.sun.jersey</groupId>
+            <artifactId>jersey-json</artifactId>
+            <version>1.18.2</version>
+        </dependency>
+        <dependency>
+            <groupId>javax.servlet</groupId>
+            <artifactId>javax.servlet-api</artifactId>
+            <version>3.1.0</version>
+            <scope>provided</scope>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..8c0b1f4
--- /dev/null
+++ b/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/ClientUtils.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..4e7f5b6
--- /dev/null
+++ b/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/ObjectMapperResolver.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..587b3d8
--- /dev/null
+++ b/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/WebUtils.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/processor-utilities/pom.xml
----------------------------------------------------------------------
diff --git a/commons/processor-utilities/pom.xml b/commons/processor-utilities/pom.xml
new file mode 100644
index 0000000..0519b7f
--- /dev/null
+++ b/commons/processor-utilities/pom.xml
@@ -0,0 +1,47 @@
+<?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-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>
+            <version>[0.0.1-SNAPSHOT, 1.0.0-SNAPSHOT)</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>[0.0.1-SNAPSHOT,1.0.0-SNAPSHOT)</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-utils</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..1f77093
--- /dev/null
+++ b/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/FlowFileFilters.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..0d66df5
--- /dev/null
+++ b/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/SSLProperties.java
@@ -0,0 +1,226 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..10748fe
--- /dev/null
+++ b/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java
@@ -0,0 +1,544 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..359def2
--- /dev/null
+++ b/commons/processor-utilities/src/test/java/org/apache/nifi/processor/TestFormatUtils.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..2ae50c9
--- /dev/null
+++ b/commons/processor-utilities/src/test/java/org/apache/nifi/processor/util/TestStandardValidators.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/remote-communications-utils/pom.xml
----------------------------------------------------------------------
diff --git a/commons/remote-communications-utils/pom.xml b/commons/remote-communications-utils/pom.xml
new file mode 100644
index 0000000..5e5ebc1
--- /dev/null
+++ b/commons/remote-communications-utils/pom.xml
@@ -0,0 +1,29 @@
+<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-parent</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>remote-communications-utils</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <packaging>jar</packaging>
+
+    <name>remote-communications-utils</name>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/StandardVersionNegotiator.java
----------------------------------------------------------------------
diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/StandardVersionNegotiator.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/StandardVersionNegotiator.java
new file mode 100644
index 0000000..77c34c9
--- /dev/null
+++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/StandardVersionNegotiator.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/VersionNegotiator.java
----------------------------------------------------------------------
diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/VersionNegotiator.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/VersionNegotiator.java
new file mode 100644
index 0000000..74f9b3d
--- /dev/null
+++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/VersionNegotiator.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/exception/TransmissionDisabledException.java
----------------------------------------------------------------------
diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/exception/TransmissionDisabledException.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/exception/TransmissionDisabledException.java
new file mode 100644
index 0000000..05fd915
--- /dev/null
+++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/exception/TransmissionDisabledException.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java
----------------------------------------------------------------------
diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java
new file mode 100644
index 0000000..71cf894
--- /dev/null
+++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 {
+
+    }
+}


[42/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..aa6f8f3
--- /dev/null
+++ b/commons/nifi-properties/src/main/java/org/apache/nifi/util/StringUtils.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/nifi-security-utils/pom.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-security-utils/pom.xml b/commons/nifi-security-utils/pom.xml
new file mode 100644
index 0000000..76e9ac1
--- /dev/null
+++ b/commons/nifi-security-utils/pom.xml
@@ -0,0 +1,40 @@
+<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-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>
+            <version>3.3.2</version>
+        </dependency>
+    </dependencies>
+</project>
+

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..087d891
--- /dev/null
+++ b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/CertificateUtils.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..741fdde
--- /dev/null
+++ b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/EncryptionMethod.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..18574bb
--- /dev/null
+++ b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/KeystoreType.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..9abfcc3
--- /dev/null
+++ b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SecurityStoreTypes.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..2371b0c
--- /dev/null
+++ b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SslContextFactory.java
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/nifi-socket-utils/pom.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-socket-utils/pom.xml b/commons/nifi-socket-utils/pom.xml
new file mode 100644
index 0000000..8e06433
--- /dev/null
+++ b/commons/nifi-socket-utils/pom.xml
@@ -0,0 +1,65 @@
+<?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-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>
+            <version>0.0.1-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-logging-utils</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-net</groupId>
+            <artifactId>commons-net</artifactId>
+            <version>3.3</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+            <version>3.3.2</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+            <version>2.4</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-properties</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..172c593
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/AbstractChannelReader.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..a413ad2
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/BufferPool.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..2ae2c07
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/ChannelDispatcher.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b0a1cfb
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/ChannelListener.java
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1eb5c7e
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/DatagramChannelReader.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..db2c102
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/SocketChannelReader.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..fce59c6
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/AbstractStreamConsumer.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..d75b7d7
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/StreamConsumer.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..df298d5
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/StreamConsumerFactory.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.io.nio.consumer;
+
+/**
+ *
+ * @author none
+ */
+public interface StreamConsumerFactory {
+
+    StreamConsumer newInstance(String streamId);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..7ed5ad4
--- /dev/null
+++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SSLContextFactory.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+    }
+}


[16/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..0d34dae
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowResponseMessage.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..0064cb6
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/HeartbeatMessage.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..c6d2d44
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/MulticastProtocolMessage.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..9237a92
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/NodeBulletinsMessage.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..ee38deb
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PingMessage.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..a289abc
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PrimaryRoleAssignmentMessage.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..6bf2a13
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ba45e28
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionFailureMessage.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..eab3d5d
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionRequestMessage.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..fd0f921
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionResponseMessage.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..92708ba
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ServiceBroadcastMessage.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..fa201bb
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..5b5816d
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b438e44
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/SocketConfigurationFactoryBean.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..07ea7a4
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml
@@ -0,0 +1,110 @@
+<?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/4d998c12/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
new file mode 100644
index 0000000..59837c1
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..91f81af
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+        }
+    }
+    
+    @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());
+        
+    }
+    
+    @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/4d998c12/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
new file mode 100644
index 0000000..b1c156b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ec1f26d
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..af00590
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.MulticastProtocolListener;
+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);
+        }
+    }
+    
+    @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/4d998c12/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
new file mode 100644
index 0000000..1c5ba9e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java
@@ -0,0 +1,203 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..07ee83a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/DelayedProtocolHandler.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+}


[51/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
Initial code contribution


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

Branch: refs/heads/master
Commit: 4d998c12c95a6e5ce3d66c0d861e75e33b5cf013
Parents: 
Author: joewitt <jo...@apache.org>
Authored: Mon Dec 8 15:22:14 2014 -0500
Committer: joewitt <jo...@apache.org>
Committed: Mon Dec 8 15:22:14 2014 -0500

----------------------------------------------------------------------
 .gitignore                                      |    9 +
 DISCLAIMER                                      |   15 +
 LICENSE                                         |  202 +
 NOTICE                                          |    7 +
 README.md                                       |   65 +
 assemblies/nifi/pom.xml                         |  485 +
 .../nifi/src/main/assembly/dependencies.xml     |  123 +
 commons/core-flowfile-attributes/pom.xml        |   32 +
 .../flowfile/attributes/CoreAttributes.java     |   72 +
 .../attributes/FlowFileAttributeKey.java        |   21 +
 commons/data-provenance-utils/.gitignore        |    2 +
 commons/data-provenance-utils/pom.xml           |   42 +
 .../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               |   43 +
 .../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/naive-search-ring-buffer/pom.xml        |   30 +
 .../apache/nifi/util/NaiveSearchRingBuffer.java |  135 +
 .../nifi/util/TestNaiveSearchRingBuffer.java    |   72 +
 commons/nifi-expression-language/pom.xml        |   59 +
 .../language/antlr/AttributeExpressionLexer.g   |  207 +
 .../language/antlr/AttributeExpressionParser.g  |  139 +
 .../output/AttributeExpressionLexer.tokens      |   72 +
 .../expression/language/EmptyPreparedQuery.java |   62 +
 .../expression/language/PreparedQuery.java      |   39 +
 .../attribute/expression/language/Query.java    | 1183 +++
 .../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      |  115 +
 .../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   |   60 +
 .../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/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 +
 .../evaluation/functions/ReplaceEvaluator.java  |   55 +
 .../functions/ReplaceNullEvaluator.java         |   47 +
 .../functions/StartsWithEvaluator.java          |   53 +
 .../functions/StringToDateEvaluator.java        |   64 +
 .../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 +
 .../selection/AllAttributesEvaluator.java       |   68 +
 .../selection/AnyAttributeEvaluator.java        |   68 +
 .../AnyMatchingAttributeEvaluator.java          |   21 +
 .../selection/DelineatedAttributeEvaluator.java |   83 +
 .../selection/MultiAttributeEvaluator.java      |   24 +
 .../selection/MultiMatchAttributeEvaluator.java |   82 +
 .../selection/MultiNamedAttributeEvaluator.java |   64 +
 .../AttributeExpressionLanguageException.java   |   34 +
 ...ibuteExpressionLanguageParsingException.java |   34 +
 .../exception/IllegalAttributeException.java    |   28 +
 .../expression/language/TestQuery.java          | 1068 +++
 .../language/TestStandardPreparedQuery.java     |   92 +
 commons/nifi-file-utils/pom.xml                 |   35 +
 .../java/org/apache/nifi/file/FileUtils.java    |  612 ++
 commons/nifi-logging-utils/pom.xml              |   35 +
 .../java/org/apache/nifi/logging/NiFiLog.java   |  367 +
 commons/nifi-parent/pom.xml                     |  217 +
 commons/nifi-properties/.gitignore              |    3 +
 commons/nifi-properties/pom.xml                 |   29 +
 .../org/apache/nifi/util/NiFiProperties.java    |  882 ++
 .../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               |   65 +
 .../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-stream-utils/.gitignore            |    1 +
 commons/nifi-stream-utils/pom.xml               |   42 +
 .../org/apache/nifi/io/BufferedInputStream.java |   37 +
 .../apache/nifi/io/BufferedOutputStream.java    |  140 +
 .../apache/nifi/io/ByteArrayInputStream.java    |  250 +
 .../apache/nifi/io/ByteArrayOutputStream.java   |  250 +
 .../apache/nifi/io/ByteCountingInputStream.java |  104 +
 .../nifi/io/ByteCountingOutputStream.java       |   63 +
 .../org/apache/nifi/io/DataOutputStream.java    |  417 +
 .../org/apache/nifi/io/GZIPOutputStream.java    |   41 +
 .../nifi/io/LeakyBucketStreamThrottler.java     |  324 +
 .../apache/nifi/io/NonCloseableInputStream.java |   56 +
 .../nifi/io/NonCloseableOutputStream.java       |   51 +
 .../org/apache/nifi/io/NullOutputStream.java    |   46 +
 .../org/apache/nifi/io/StreamThrottler.java     |   33 +
 .../java/org/apache/nifi/io/StreamUtils.java    |  257 +
 .../org/apache/nifi/io/ZipOutputStream.java     |   38 +
 .../exception/BytePatternNotFoundException.java |   28 +
 .../io/util/NonThreadSafeCircularBuffer.java    |   69 +
 .../nifi/io/TestLeakyBucketThrottler.java       |  144 +
 .../src/test/resources/logback-test.xml         |   48 +
 commons/nifi-utils/.gitignore                   |    8 +
 commons/nifi-utils/pom.xml                      |   28 +
 .../apache/nifi/io/CompoundUpdateMonitor.java   |  115 +
 .../org/apache/nifi/io/LastModifiedMonitor.java |   30 +
 .../java/org/apache/nifi/io/MD5SumMonitor.java  |   49 +
 .../apache/nifi/io/SynchronousFileWatcher.java  |  123 +
 .../java/org/apache/nifi/io/UpdateMonitor.java  |   25 +
 .../org/apache/nifi/util/BooleanHolder.java     |   25 +
 .../java/org/apache/nifi/util/FormatUtils.java  |  204 +
 .../org/apache/nifi/util/IntegerHolder.java     |   54 +
 .../java/org/apache/nifi/util/LongHolder.java   |   60 +
 .../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 +
 .../nifi/util/timebuffer/EntityAccess.java      |   26 +
 .../nifi/util/timebuffer/LongEntityAccess.java  |   43 +
 .../nifi/util/timebuffer/TimedBuffer.java       |  114 +
 .../nifi/util/timebuffer/TimestampedLong.java   |   35 +
 .../timebuffer/TestCompoundUpdateMonitor.java   |   75 +
 .../nifi/util/timebuffer/TestRingBuffer.java    |  182 +
 .../timebuffer/TestSynchronousFileWatcher.java  |   64 +
 .../nifi/util/timebuffer/TestTimedBuffer.java   |  106 +
 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/processor-utilities/pom.xml             |   47 +
 .../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/remote-communications-utils/pom.xml     |   29 +
 .../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 +
 .../io/TestCompressionInputOutputStreams.java   |  153 +
 commons/search-utils/pom.xml                    |   32 +
 .../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 +
 commons/wali/.gitignore                         |    2 +
 commons/wali/pom.xml                            |   42 +
 .../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 +
 extensions/file-authorization-provider/pom.xml  |   90 +
 .../FileAuthorizationProvider.java              |  568 ++
 ....apache.nifi.authorization.AuthorityProvider |   15 +
 .../src/main/xsd/users.xsd                      |   64 +
 .../FileAuthorizationProviderTest.java          |  127 +
 misc/build-order.sh                             |   79 +
 misc/nar-maven-plugin/pom.xml                   |   83 +
 .../src/main/java/nifi/NarMojo.java             |  610 ++
 .../resources/META-INF/plexus/components.xml    |   52 +
 .../distributed-cache-client-service/pom.xml    |   67 +
 .../distributed/cache/client/CommsSession.java  |   46 +
 .../DistributedMapCacheClientService.java       |  301 +
 .../DistributedSetCacheClientService.java       |  212 +
 .../cache/client/SSLCommsSession.java           |  106 +
 .../cache/client/StandardCommsSession.java      |  124 +
 ...org.apache.nifi.controller.ControllerService |   16 +
 .../index.html                                  |   78 +
 .../index.html                                  |    0
 .../distributed-cache-protocol/pom.xml          |   39 +
 .../cache/protocol/ProtocolHandshake.java       |  119 +
 .../protocol/exception/HandshakeException.java  |   27 +
 .../distributed-cache-server/pom.xml            |   81 +
 .../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       |  530 +
 .../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      |   49 +
 .../distributed-cache-services-bundle/pom.xml   |   83 +
 .../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  |  148 +
 .../service/action/AbstractUserAction.java      |   97 +
 .../admin/service/action/AddActionsAction.java  |   48 +
 .../service/action/AdministrationAction.java    |   38 +
 .../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 |  629 ++
 .../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           |  516 +
 ...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     |  154 +
 .../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 +
 .../impl/NiFiAuthorizationServiceTest.java      |  284 +
 .../framework/client-dto/.gitignore             |    6 +
 .../framework/client-dto/pom.xml                |   47 +
 .../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  |   45 +
 .../nifi/web/api/dto/util/TimeAdapter.java      |   45 +
 .../nifi/web/api/dto/util/TimestampAdapter.java |   45 +
 .../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              |  381 +
 .../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       |  133 +
 .../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  |  133 +
 .../cluster/client/MulticastTestClient.java     |  149 +
 .../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 |  551 ++
 .../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 | 3616 +++++++
 .../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  |   99 +
 .../impl/DataFlowManagementServiceImplTest.java |  341 +
 .../impl/HttpRequestReplicatorImplTest.java     |  368 +
 .../impl/HttpResponseMapperImplTest.java        |  126 +
 .../manager/impl/TestWebClusterManager.java     |   52 +
 .../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     |  170 +
 .../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 |   60 +
 .../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  | 1093 +++
 .../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  |  253 +
 .../RemoteProcessGroupPortDescriptor.java       |   92 +
 .../org/apache/nifi/logging/LogMessage.java     |   73 +
 .../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     |  130 +
 .../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  |  709 ++
 .../apache/nifi/controller/FlowController.java  | 3534 +++++++
 .../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        | 1345 +++
 .../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      | 2563 +++++
 .../StandardProcessSessionFactory.java          |   33 +
 .../repository/StandardProvenanceReporter.java  |  433 +
 .../repository/StandardRepositoryRecord.java    |  196 +
 .../StandardRepositoryStatusReport.java         |   88 +
 .../repository/VolatileContentRepository.java   |  642 ++
 .../repository/VolatileFlowFileRepository.java  |  109 +
 .../WriteAheadFlowFileRepository.java           |  861 ++
 .../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  |  326 +
 .../scheduling/ProcessContextFactory.java       |   51 +
 .../scheduling/QuartzSchedulingAgent.java       |  220 +
 .../controller/scheduling/ScheduleState.java    |   90 +
 .../controller/scheduling/SchedulingAgent.java  |   45 +
 .../scheduling/StandardProcessScheduler.java    |  566 ++
 .../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 |  120 +
 .../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    |  983 ++
 .../nifi/groups/StandardProcessGroup.java       | 2015 ++++
 .../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 | 1612 ++++
 ...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   |  149 +
 .../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  | 1018 ++
 .../TestStandardProvenanceReporter.java         |   65 +
 .../TestVolatileContentRepository.java          |  183 +
 .../TestWriteAheadFlowFileRepository.java       |  135 +
 .../repository/io/TestLimitedOutputStream.java  |   77 +
 .../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
 .../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     |  205 +
 .../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  |   50 +
 .../framework/resources/pom.xml                 |   49 +
 .../src/main/assembly/dependencies.xml          |   36 +
 .../resources/src/main/resources/DISCLAIMER     |   15 +
 .../resources/src/main/resources/LICENSE        |  202 +
 .../resources/src/main/resources/NOTICE         |    7 +
 .../resources/src/main/resources/bin/nifi.sh    |  350 +
 .../main/resources/conf/authority-providers.xml |   43 +
 .../main/resources/conf/authorized-users.xml    |   57 +
 .../main/resources/conf/controller-services.xml |   18 +
 .../src/main/resources/conf/logback.xml         |  114 +
 .../src/main/resources/conf/nifi.properties     |  159 +
 .../src/main/resources/conf/reporting-tasks.xml |   49 +
 .../resources/src/main/resources/docs/README.md |   65 +
 .../framework/runtime/.gitignore                |    2 +
 .../framework-bundle/framework/runtime/pom.xml  |   36 +
 .../src/main/java/org/apache/nifi/NiFi.java     |  178 +
 .../main/java/org/apache/nifi/NiFiServer.java   |   31 +
 .../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              |   80 +
 .../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   |  332 +
 .../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            |  139 +
 .../src/main/assembly/distribution.xml          |   42 +
 .../org/apache/nifi/web/server/JettyServer.java |  682 ++
 .../server/ServerConfigurationException.java    |   36 +
 .../org/apache/nifi/web/webdefault.xml          |  556 ++
 .../framework/web/nifi-web-api/.gitignore       |    1 +
 .../web/nifi-web-api/nb-configuration.xml       |   18 +
 .../framework/web/nifi-web-api/pom.xml          |  302 +
 .../nifi-web-api/src/main/enunciate/default.css |  266 +
 .../src/main/enunciate/enunciate.xml            |   34 +
 .../src/main/enunciate/images/bgcode.gif        |  Bin 0 -> 56 bytes
 .../src/main/enunciate/images/bgcontainer.png   |  Bin 0 -> 386 bytes
 .../src/main/enunciate/images/bgul.gif          |  Bin 0 -> 304 bytes
 .../src/main/enunciate/images/header.png        |  Bin 0 -> 51928 bytes
 .../src/main/enunciate/images/li.png            |  Bin 0 -> 191 bytes
 .../src/main/enunciate/images/quote.gif         |  Bin 0 -> 228 bytes
 .../src/main/enunciate/images/search.gif        |  Bin 0 -> 680 bytes
 .../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 | 2141 +++++
 .../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 |   62 +
 .../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   | 1339 +++
 .../web/controller/StandardSearchContext.java   |   80 +
 .../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 |  302 +
 .../src/main/webapp/WEB-INF/web.xml             |   99 +
 .../apache/nifi/integration/NiFiWebApiTest.java |  274 +
 .../accesscontrol/AdminAccessControlTest.java   | 1090 +++
 .../accesscontrol/DfmAccessControlTest.java     | 1386 +++
 .../ReadOnlyAccessControlTest.java              | 1064 +++
 .../util/NiFiTestAuthorizationProvider.java     |  172 +
 .../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  |   88 +
 .../main/webapp/WEB-INF/jsp/documentation.jsp   |  114 +
 .../WEB-INF/jsp/no-documentation-found.jsp      |   31 +
 .../src/main/webapp/WEB-INF/web.xml             |   33 +
 .../src/main/webapp/css/component-usage.css     |   71 +
 .../nifi-web-docs/src/main/webapp/css/main.css  |  203 +
 .../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           |  275 +
 .../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    |   36 +
 .../framework/web/nifi-web-ui/.gitignore        |    1 +
 .../framework/web/nifi-web-ui/pom.xml           |  655 ++
 .../apache/nifi/web/filter/IeEdgeHeader.java    |   57 +
 .../org/apache/nifi/web/servlet/ConvertSvg.java |  115 +
 .../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     |   49 +
 .../src/main/webapp/WEB-INF/pages/canvas.jsp    |  118 +
 .../src/main/webapp/WEB-INF/pages/cluster.jsp   |   61 +
 .../src/main/webapp/WEB-INF/pages/counters.jsp  |   60 +
 .../src/main/webapp/WEB-INF/pages/history.jsp   |   64 +
 .../main/webapp/WEB-INF/pages/message-page.jsp  |   34 +
 .../main/webapp/WEB-INF/pages/provenance.jsp    |   67 +
 .../src/main/webapp/WEB-INF/pages/summary.jsp   |   72 +
 .../src/main/webapp/WEB-INF/pages/templates.jsp |   62 +
 .../src/main/webapp/WEB-INF/pages/users.jsp     |   70 +
 .../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   |   55 +
 .../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 |  209 +
 .../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        |   81 +
 .../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         |   23 +
 .../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 |  148 +
 .../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   |  265 +
 .../nifi-web-ui/src/main/webapp/css/header.css  |  676 ++
 .../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    |  279 +
 .../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  |  180 +
 .../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 |  366 +
 .../src/main/webapp/css/processor-details.css   |  119 +
 .../src/main/webapp/css/provenance.css          |  644 ++
 .../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 |  757 ++
 .../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 -> 90275 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 -> 1682 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 -> 7854 bytes
 .../src/main/webapp/images/nifi16.ico           |  Bin 0 -> 1150 bytes
 .../src/main/webapp/images/nifi_about.png       |  Bin 0 -> 18958 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/edit/matchbrackets.js   |  125 +
 .../js/codemirror/addon/hint/show-hint.css      |   38 +
 .../js/codemirror/addon/hint/show-hint.js       |  411 +
 .../webapp/js/codemirror/lib/codemirror.css     |  270 +
 .../main/webapp/js/codemirror/lib/codemirror.js | 8488 +++++++++++++++++
 .../nifi-web-ui/src/main/webapp/js/d3/LICENSE   |   26 +
 .../nifi-web-ui/src/main/webapp/js/d3/d3.min.js | 5385 +++++++++++
 .../webapp/js/jquery/combo/jquery.combo.css     |   82 +
 .../main/webapp/js/jquery/combo/jquery.combo.js |  292 +
 .../main/webapp/js/jquery/combo/menuArrow.png   |  Bin 0 -> 251 bytes
 .../webapp/js/jquery/combo/menuArrowSmall.png   |  Bin 0 -> 233 bytes
 .../images/ui-bg_flat_0_aaaaaa_40x100.png       |  Bin 0 -> 180 bytes
 .../images/ui-bg_flat_75_ffffff_40x100.png      |  Bin 0 -> 178 bytes
 .../images/ui-bg_glass_55_fbf9ee_1x400.png      |  Bin 0 -> 120 bytes
 .../images/ui-bg_glass_65_ffffff_1x400.png      |  Bin 0 -> 105 bytes
 .../images/ui-bg_glass_75_dadada_1x400.png      |  Bin 0 -> 111 bytes
 .../images/ui-bg_glass_75_e6e6e6_1x400.png      |  Bin 0 -> 110 bytes
 .../images/ui-bg_glass_95_fef1ec_1x400.png      |  Bin 0 -> 119 bytes
 .../ui-bg_highlight-soft_75_cccccc_1x100.png    |  Bin 0 -> 101 bytes
 .../images/ui-icons_222222_256x240.png          |  Bin 0 -> 4369 bytes
 .../images/ui-icons_2e83ff_256x240.png          |  Bin 0 -> 4369 bytes
 .../images/ui-icons_454545_256x240.png          |  Bin 0 -> 8543 bytes
 .../images/ui-icons_888888_256x240.png          |  Bin 0 -> 4369 bytes
 .../images/ui-icons_cd0a0a_256x240.png          |  Bin 0 -> 4369 bytes
 .../images/ui-icons_ffffff_256x240.png          |  Bin 0 -> 8358 bytes
 .../css/smoothness/jquery-ui-1.8.10.custom.css  |  625 ++
 .../src/main/webapp/js/jquery/jquery-1.7.js     | 9017 ++++++++++++++++++
 .../src/main/webapp/js/jquery/jquery-1.7.min.js | 2934 ++++++
 .../js/jquery/jquery-ui-1.8.10.custom.min.js    | 6354 ++++++++++++
 .../src/main/webapp/js/jquery/jquery.center.js  |   27 +
 .../src/main/webapp/js/jquery/jquery.count.js   |   91 +
 .../src/main/webapp/js/jquery/jquery.each.js    |   31 +
 .../main/webapp/js/jquery/jquery.ellipsis.js    |  155 +
 .../js/jquery/jquery.event.drag-2.0.min.js      |  194 +
 .../src/main/webapp/js/jquery/jquery.form.js    | 1118 +++
 .../src/main/webapp/js/jquery/jquery.tab.js     |   69 +
 .../js/jquery/minicolors/jquery.minicolors.css  |  245 +
 .../jquery/minicolors/jquery.minicolors.min.js  |  420 +
 .../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       |  306 +
 .../webapp/js/jquery/nfeditor/languages/nfel.js |  630 ++
 .../main/webapp/js/jquery/qtip2/jquery.qtip.css |  557 ++
 .../webapp/js/jquery/qtip2/jquery.qtip.min.js   |  706 ++
 .../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       |  121 +
 .../js/jquery/slickgrid/css/slick.grid.css      |  158 +
 .../slickgrid/plugins/slick.autotooltips.js     |   48 +
 .../plugins/slick.cellrangedecorator.js         |   64 +
 .../plugins/slick.cellrangeselector.js          |  111 +
 .../plugins/slick.cellselectionmodel.js         |   90 +
 .../plugins/slick.rowselectionmodel.js          |  184 +
 .../webapp/js/jquery/slickgrid/slick.core.js    |  423 +
 .../js/jquery/slickgrid/slick.dataview.js       |  914 ++
 .../webapp/js/jquery/slickgrid/slick.editors.js |  512 +
 .../js/jquery/slickgrid/slick.formatters.js     |   55 +
 .../webapp/js/jquery/slickgrid/slick.grid.js    | 2832 ++++++
 .../webapp/js/jquery/tabbs/jquery.tabbs.css     |   48 +
 .../main/webapp/js/jquery/tabbs/jquery.tabbs.js |   96 +
 .../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  | 1119 +++
 .../src/main/webapp/js/nf/canvas/nf-birdseye.js |  347 +
 .../webapp/js/nf/canvas/nf-canvas-header.js     |  293 +
 .../webapp/js/nf/canvas/nf-canvas-toolbar.js    |  158 +
 .../webapp/js/nf/canvas/nf-canvas-toolbox.js    | 1203 +++
 .../main/webapp/js/nf/canvas/nf-canvas-utils.js | 1283 +++
 .../src/main/webapp/js/nf/canvas/nf-canvas.js   | 1462 +++
 .../main/webapp/js/nf/canvas/nf-clipboard.js    |   92 +
 .../main/webapp/js/nf/canvas/nf-connectable.js  |  209 +
 .../js/nf/canvas/nf-connection-configuration.js | 1375 +++
 .../main/webapp/js/nf/canvas/nf-connection.js   | 1456 +++
 .../main/webapp/js/nf/canvas/nf-context-menu.js |  442 +
 .../js/nf/canvas/nf-custom-processor-ui.js      |   43 +
 .../main/webapp/js/nf/canvas/nf-draggable.js    |  295 +
 .../src/main/webapp/js/nf/canvas/nf-funnel.js   |  271 +
 .../src/main/webapp/js/nf/canvas/nf-go-to.js    |  720 ++
 .../webapp/js/nf/canvas/nf-graph-control.js     |  113 +
 .../src/main/webapp/js/nf/canvas/nf-graph.js    |  188 +
 .../js/nf/canvas/nf-label-configuration.js      |  130 +
 .../src/main/webapp/js/nf/canvas/nf-label.js    |  549 ++
 .../js/nf/canvas/nf-port-configuration.js       |  166 +
 .../main/webapp/js/nf/canvas/nf-port-details.js |   59 +
 .../src/main/webapp/js/nf/canvas/nf-port.js     |  622 ++
 .../nf/canvas/nf-process-group-configuration.js |  103 +
 .../js/nf/canvas/nf-process-group-details.js    |   57 +
 .../webapp/js/nf/canvas/nf-process-group.js     | 1040 ++
 .../js/nf/canvas/nf-processor-configuration.js  |  703 ++
 .../nf-processor-property-combo-editor.js       |  170 +
 .../canvas/nf-processor-property-nfel-editor.js |  212 +
 .../js/nf/canvas/nf-processor-property-table.js |  547 ++
 .../canvas/nf-processor-property-text-editor.js |  216 +
 .../main/webapp/js/nf/canvas/nf-processor.js    |  820 ++
 .../main/webapp/js/nf/canvas/nf-registration.js |   68 +
 .../nf-remote-process-group-configuration.js    |  120 +
 .../canvas/nf-remote-process-group-details.js   |   63 +
 .../nf/canvas/nf-remote-process-group-ports.js  |  516 +
 .../js/nf/canvas/nf-remote-process-group.js     | 1050 ++
 .../src/main/webapp/js/nf/canvas/nf-search.js   |  180 +
 .../nf/canvas/nf-secure-port-configuration.js   |  384 +
 .../js/nf/canvas/nf-secure-port-details.js      |  114 +
 .../main/webapp/js/nf/canvas/nf-selectable.js   |   56 +
 .../src/main/webapp/js/nf/canvas/nf-settings.js |  145 +
 .../src/main/webapp/js/nf/canvas/nf-snippet.js  |  206 +
 .../src/main/webapp/js/nf/canvas/nf-storage.js  |  133 +
 .../webapp/js/nf/canvas/nf-toolbar-action.js    |   82 +
 .../webapp/js/nf/cluster/nf-cluster-table.js    |  589 ++
 .../src/main/webapp/js/nf/cluster/nf-cluster.js |  154 +
 .../webapp/js/nf/counters/nf-counters-table.js  |  291 +
 .../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    |  426 +
 .../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          |  843 ++
 .../main/webapp/js/nf/nf-connection-details.js  |  459 +
 .../src/main/webapp/js/nf/nf-dialog.js          |  127 +
 .../src/main/webapp/js/nf/nf-namespace.js       |   20 +
 .../main/webapp/js/nf/nf-processor-details.js   |  560 ++
 .../src/main/webapp/js/nf/nf-shell.js           |  156 +
 .../src/main/webapp/js/nf/nf-status-history.js  | 1347 +++
 .../js/nf/provenance/nf-provenance-lineage.js   | 1412 +++
 .../js/nf/provenance/nf-provenance-table.js     | 1365 +++
 .../webapp/js/nf/provenance/nf-provenance.js    |  204 +
 .../webapp/js/nf/summary/nf-cluster-search.js   |  168 +
 .../webapp/js/nf/summary/nf-summary-table.js    | 2393 +++++
 .../src/main/webapp/js/nf/summary/nf-summary.js |  174 +
 .../js/nf/templates/nf-templates-table.js       |  328 +
 .../main/webapp/js/nf/templates/nf-templates.js |  235 +
 .../main/webapp/js/nf/users/nf-users-table.js   | 1080 +++
 .../src/main/webapp/js/nf/users/nf-users.js     |  149 +
 .../framework-bundle/framework/web/pom.xml      |   85 +
 .../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   |   38 +
 .../anonymous/NiFiAnonymousUserFilter.java      |  101 +
 .../NiFiAuthenticationEntryPoint.java           |   68 +
 .../authorization/NiFiAuthorizationService.java |  150 +
 .../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 +
 nar-bundles/framework-bundle/nar/.gitignore     |    1 +
 nar-bundles/framework-bundle/nar/pom.xml        |  113 +
 nar-bundles/framework-bundle/pom.xml            |  477 +
 .../hadoop-bundle/hdfs-processors/pom.xml       |   77 +
 .../hadoop/AbstractHadoopProcessor.java         |  221 +
 .../hadoop/CreateHadoopSequenceFile.java        |  177 +
 ...lowFileStreamUnpackerSequenceFileWriter.java |  170 +
 .../apache/nifi/processors/hadoop/GetHDFS.java  |  562 ++
 .../processors/hadoop/GetHDFSSequenceFile.java  |  146 +
 .../nifi/processors/hadoop/KeyValueReader.java  |  132 +
 .../apache/nifi/processors/hadoop/PutHDFS.java  |  402 +
 .../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     |  125 +
 .../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
 nar-bundles/hadoop-bundle/nar/pom.xml           |   41 +
 nar-bundles/hadoop-bundle/pom.xml               |  106 +
 nar-bundles/hadoop-libraries-bundle/nar/pom.xml |   52 +
 nar-bundles/hadoop-libraries-bundle/pom.xml     |   60 +
 nar-bundles/jetty-bundle/pom.xml                |   82 +
 .../monitor-threshold-bundle/nar/pom.xml        |   40 +
 nar-bundles/monitor-threshold-bundle/pom.xml    |  111 +
 .../monitor-threshold-bundle/processor/pom.xml  |   53 +
 .../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  |    1 +
 .../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
 .../js/jquery/jqgrid/css/ellipsis-xbl.xml       |   29 +
 .../webapp/js/jquery/jqgrid/css/ui.jqgrid.css   |  212 +
 .../js/jquery/jqgrid/js/i18n/grid.locale-en.js  |    5 +
 .../js/jquery/jqgrid/js/jquery.jqGrid.min.js    |  416 +
 .../ui/src/main/webapp/js/nf-common.js          |  887 ++
 .../ui/src/main/xsd/threshold_settings.xsd      |   54 +
 nar-bundles/nar-container-common/pom.xml        |  121 +
 .../nar/pom.xml                                 |   36 +
 .../persistent-provenance-repository/pom.xml    |   67 +
 .../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     | 1129 +++
 .../pom.xml                                     |   82 +
 nar-bundles/ssl-context-bundle/nar/pom.xml      |   39 +
 nar-bundles/ssl-context-bundle/pom.xml          |   49 +
 .../ssl-context-service/pom.xml                 |   51 +
 .../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
 .../standard-bundle/jms-processors/pom.xml      |   63 +
 .../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    |  463 +
 .../nifi/processors/jms/util/JmsProperties.java |  177 +
 .../jms/util/WrappedMessageConsumer.java        |   77 +
 .../jms/util/WrappedMessageProducer.java        |   77 +
 .../org.apache.nifi.processor.Processor         |   17 +
 .../index.html                                  |  119 +
 .../index.html                                  |  123 +
 .../index.html                                  |  153 +
 .../nifi/processors/jms/GetJMSQueueTest.java    |  141 +
 nar-bundles/standard-bundle/nar/pom.xml         |   61 +
 nar-bundles/standard-bundle/pom.xml             |  105 +
 .../standard-ganglia-reporter/pom.xml           |   53 +
 .../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 |   67 +
 .../prioritizer/OldestFirstPrioritizerTest.java |   67 +
 .../PriorityAttributePrioritizerTest.java       |  117 +
 .../standard-bundle/standard-processors/pom.xml |  192 +
 .../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 |  404 +
 .../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       |  443 +
 .../processors/standard/GetFileTransfer.java    |  300 +
 .../nifi/processors/standard/GetHTTP.java       |  447 +
 .../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    |  310 +
 .../nifi/processors/standard/ListenUDP.java     |  627 ++
 .../nifi/processors/standard/LogAttribute.java  |  262 +
 .../nifi/processors/standard/MergeContent.java  |  974 ++
 .../nifi/processors/standard/ModifyBytes.java   |  134 +
 .../processors/standard/MonitorActivity.java    |  206 +
 .../nifi/processors/standard/PostHTTP.java      |  859 ++
 .../nifi/processors/standard/PutEmail.java      |  297 +
 .../apache/nifi/processors/standard/PutFTP.java |  141 +
 .../nifi/processors/standard/PutFile.java       |  366 +
 .../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    |  308 +
 .../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                                  |  184 +
 .../index.html                                  |  143 +
 .../index.html                                  |  250 +
 .../index.html                                  |   88 +
 .../index.html                                  |   89 +
 .../index.html                                  |  136 +
 .../index.html                                  |  181 +
 .../index.html                                  |   80 +
 .../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 +
 .../standard/RESTServiceContentModified.java    |   77 +
 .../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   |  185 +
 .../nifi/processors/standard/TestGetHTTP.java   |  262 +
 .../processors/standard/TestHashAttribute.java  |   99 +
 .../processors/standard/TestHashContent.java    |   67 +
 .../standard/TestIdentifyMimeType.java          |  131 +
 .../processors/standard/TestInvokeHTTP.java     |  622 ++
 .../nifi/processors/standard/TestListenUDP.java |  214 +
 .../processors/standard/TestMergeContent.java   |  556 ++
 .../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    |   87 +
 .../processors/standard/TestSegmentContent.java |   65 +
 .../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            |   67 +
 .../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 +
 .../load-distribution-service-api/pom.xml       |   36 +
 .../nifi/loading/LoadDistributionListener.java  |   24 +
 .../nifi/loading/LoadDistributionService.java   |   33 +
 .../standard-services-api-bundle/pom.xml        |   74 +
 .../ssl-context-service-api/pom.xml             |   36 +
 .../org/apache/nifi/ssl/SSLContextService.java  |   55 +
 .../standard-services-api-nar/pom.xml           |   44 +
 .../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 |   51 +
 nar-bundles/update-attribute-bundle/pom.xml     |   69 +
 .../update-attribute-bundle/processor/pom.xml   |   55 +
 .../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  |   97 +
 .../attributes/UpdateAttributeModelFactory.java |  127 +
 .../attributes/api/ObjectMapperResolver.java    |   51 +
 .../update/attributes/api/RuleResource.java     |  683 ++
 .../nifi/update/attributes/dto/ActionDTO.java   |   66 +
 .../update/attributes/dto/ConditionDTO.java     |   57 +
 .../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  |    1 +
 .../src/main/webapp/WEB-INF/jsp/worksheet.jsp   |  189 +
 .../ui/src/main/webapp/WEB-INF/web.xml          |   46 +
 .../ui/src/main/webapp/css/main.css             |  411 +
 .../ui/src/main/webapp/css/reset.css            |   59 +
 .../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        | 1886 ++++
 .../nar/pom.xml                                 |   36 +
 .../pom.xml                                     |   48 +
 .../volatile-provenance-repository/pom.xml      |   47 +
 .../VolatileProvenanceRepository.java           |  743 ++
 ...he.nifi.provenance.ProvenanceEventRepository |   15 +
 .../TestVolatileProvenanceRepository.java       |  178 +
 .../src/test/resources/nifi.properties          |   18 +
 nifi-api/.gitignore                             |    2 +
 nifi-api/pom.xml                                |   32 +
 .../apache/nifi/authorization/Authority.java    |   97 +
 .../nifi/authorization/AuthorityProvider.java   |  162 +
 .../AuthorityProviderConfigurationContext.java  |   52 +
 .../AuthorityProviderInitializationContext.java |   27 +
 .../authorization/AuthorityProviderLookup.java  |   25 +
 .../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           |  308 +
 .../controller/repository/FlowFileRecord.java   |   54 +
 .../repository/FlowFileRepository.java          |  128 +
 .../repository/FlowFileSwapManager.java         |   60 +
 .../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 +
 .../org/apache/nifi/processor/TestDataUnit.java |   44 +
 nifi-api/src/test/resources/logback-test.xml    |   27 +
 nifi-mock/pom.xml                               |   64 +
 .../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  |  485 +
 .../java/org/apache/nifi/util/TestRunner.java   |  531 ++
 .../java/org/apache/nifi/util/TestRunners.java  |   37 +
 2352 files changed, 350029 insertions(+)
----------------------------------------------------------------------



[24/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..9499c2e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeDTO.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..3cad8d8
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeEventDTO.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..8c83331
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsDTO.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..2a372f4
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PortDTO.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ab077f3
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PositionDTO.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..fb33c67
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PreviousValueDTO.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..4140046
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1481b0f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java
@@ -0,0 +1,486 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..71ba4ed
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.java
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..2741116
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..064ad21
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyHistoryDTO.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..7042aaa
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RelationshipDTO.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1e5356d
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupContentsDTO.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..df59b13
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupDTO.java
@@ -0,0 +1,279 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..7948dad
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupPortDTO.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..e608a7e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..2ee1310
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/SnippetDTO.java
@@ -0,0 +1,239 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+
+}


[14/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..702f081
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java
@@ -0,0 +1,551 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.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.file.FileUtils;
+import org.apache.nifi.io.BufferedInputStream;
+import org.apache.nifi.io.BufferedOutputStream;
+import org.apache.nifi.io.ByteArrayInputStream;
+import org.apache.nifi.io.StreamUtils;
+import org.apache.nifi.logging.NiFiLog;
+
+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.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
+                    FileUtils.syncWithRestore(primaryFlowStateFile, restoreFlowStateFile, logger);
+                }
+
+            }
+        } catch (final IOException | IllegalArgumentException | IllegalStateException | JAXBException ex) {
+            throw new DaoException(ex);
+        }
+
+    }
+
+    @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/4d998c12/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
new file mode 100644
index 0000000..e135af3
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java
@@ -0,0 +1,356 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..0fcac8c
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..2cf5812
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpClusterManager.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..fb57622
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpRequestReplicator.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..843a666
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpResponseMapper.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..3f966e5
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java
@@ -0,0 +1,329 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..49bcd35
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/BlockedByFirewallException.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..3bf9752
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ClusterException.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..365b5f0
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ConnectingNodeMutableRequestException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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);
+    }
+}


[18/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..1edcb91
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeProtocolSender.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b614e76
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolContext.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f11ad84
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolException.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..6de87db
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolHandler.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..32f0f5d
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolListener.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..bb436e0
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageMarshaller.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..c690e7b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageUnmarshaller.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..c2d16fc
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/StandardDataFlow.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..41c74eb
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/UnknownServiceAddressException.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ceb3fcb
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java
@@ -0,0 +1,245 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..933e5fa
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..24e51e0
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..bebfde8
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..e9e7d5b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..680df65
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/CopyingInputStream.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..d3764b3
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+            }
+        }
+    }
+}


[20/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..666dc75
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/CounterEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..bee7669
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/CountersEntity.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..6abcbf3
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/Entity.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..a15cc51
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FlowSnippetEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..8b43fb8
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FunnelEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..2b8cbc8
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FunnelsEntity.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..f817cce
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/HistoryEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..9fec9d8
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/InputPortEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..8637596
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/InputPortsEntity.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..867d781
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LabelEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..063eeaf
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LabelsEntity.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..5d443af
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LineageEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ac5b306
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..d7471da
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeStatusEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..59eafa1
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeSystemDiagnosticsEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..07fdab7
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/OutputPortEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..9fa398d
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/OutputPortsEntity.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..2ddddd8
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/PrioritizerTypesEntity.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..c677ef1
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f0a6e0f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupStatusEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1fdf23f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupsEntity.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..21ad431
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..19166f7
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..23237fe
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorTypesEntity.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..4962b38
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorsEntity.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..72a8528
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEntity.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..0aa0a55
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEventEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..01591cb
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceOptionsEntity.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..62a84d6
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f310b5e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupPortEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..a04c789
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupsEntity.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..6b9a88b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SearchResultsEntity.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..a9b05a1
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SnippetEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..d3c2247
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/StatusHistoryEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..e7baa16
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SystemDiagnosticsEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..8e9f07a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/TemplateEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..3400045
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/TemplatesEntity.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..963e853
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+
+}


[37/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/CompressionOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/CompressionOutputStream.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/CompressionOutputStream.java
new file mode 100644
index 0000000..bc46b0f
--- /dev/null
+++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/CompressionOutputStream.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/InterruptableInputStream.java
----------------------------------------------------------------------
diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/InterruptableInputStream.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/InterruptableInputStream.java
new file mode 100644
index 0000000..e03dfbf
--- /dev/null
+++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/InterruptableInputStream.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/InterruptableOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/InterruptableOutputStream.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/InterruptableOutputStream.java
new file mode 100644
index 0000000..cba5be6
--- /dev/null
+++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/InterruptableOutputStream.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/BufferStateManager.java
----------------------------------------------------------------------
diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/BufferStateManager.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/BufferStateManager.java
new file mode 100644
index 0000000..68913bd
--- /dev/null
+++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/BufferStateManager.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java
----------------------------------------------------------------------
diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java
new file mode 100644
index 0000000..32a3f26
--- /dev/null
+++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutputStream.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutputStream.java
new file mode 100644
index 0000000..77049ad
--- /dev/null
+++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutputStream.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
----------------------------------------------------------------------
diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
new file mode 100644
index 0000000..5810488
--- /dev/null
+++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
@@ -0,0 +1,602 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInputStream.java
----------------------------------------------------------------------
diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInputStream.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInputStream.java
new file mode 100644
index 0000000..154bd08
--- /dev/null
+++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInputStream.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutputStream.java
----------------------------------------------------------------------
diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutputStream.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutputStream.java
new file mode 100644
index 0000000..ce4e420
--- /dev/null
+++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutputStream.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/remote-communications-utils/src/test/java/org/apache/nifi/remote/io/TestCompressionInputOutputStreams.java
----------------------------------------------------------------------
diff --git a/commons/remote-communications-utils/src/test/java/org/apache/nifi/remote/io/TestCompressionInputOutputStreams.java b/commons/remote-communications-utils/src/test/java/org/apache/nifi/remote/io/TestCompressionInputOutputStreams.java
new file mode 100644
index 0000000..bd30a96
--- /dev/null
+++ b/commons/remote-communications-utils/src/test/java/org/apache/nifi/remote/io/TestCompressionInputOutputStreams.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/search-utils/pom.xml
----------------------------------------------------------------------
diff --git a/commons/search-utils/pom.xml b/commons/search-utils/pom.xml
new file mode 100644
index 0000000..569958f
--- /dev/null
+++ b/commons/search-utils/pom.xml
@@ -0,0 +1,32 @@
+<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-parent</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-search-utils</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <packaging>jar</packaging>
+
+    <name>search-utils</name>
+
+    <dependencies>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/search-utils/src/main/java/org/apache/nifi/util/search/Search.java
----------------------------------------------------------------------
diff --git a/commons/search-utils/src/main/java/org/apache/nifi/util/search/Search.java b/commons/search-utils/src/main/java/org/apache/nifi/util/search/Search.java
new file mode 100644
index 0000000..59b444a
--- /dev/null
+++ b/commons/search-utils/src/main/java/org/apache/nifi/util/search/Search.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/search-utils/src/main/java/org/apache/nifi/util/search/SearchTerm.java
----------------------------------------------------------------------
diff --git a/commons/search-utils/src/main/java/org/apache/nifi/util/search/SearchTerm.java b/commons/search-utils/src/main/java/org/apache/nifi/util/search/SearchTerm.java
new file mode 100644
index 0000000..62de964
--- /dev/null
+++ b/commons/search-utils/src/main/java/org/apache/nifi/util/search/SearchTerm.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/AhoCorasick.java
----------------------------------------------------------------------
diff --git a/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/AhoCorasick.java b/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/AhoCorasick.java
new file mode 100644
index 0000000..3b8afaf
--- /dev/null
+++ b/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/AhoCorasick.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/Node.java
----------------------------------------------------------------------
diff --git a/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/Node.java b/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/Node.java
new file mode 100644
index 0000000..0ac325c
--- /dev/null
+++ b/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/Node.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/SearchState.java
----------------------------------------------------------------------
diff --git a/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/SearchState.java b/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/SearchState.java
new file mode 100644
index 0000000..6d36ad0
--- /dev/null
+++ b/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/SearchState.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/wali/.gitignore
----------------------------------------------------------------------
diff --git a/commons/wali/.gitignore b/commons/wali/.gitignore
new file mode 100755
index 0000000..19f2e00
--- /dev/null
+++ b/commons/wali/.gitignore
@@ -0,0 +1,2 @@
+/target
+/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/wali/pom.xml
----------------------------------------------------------------------
diff --git a/commons/wali/pom.xml b/commons/wali/pom.xml
new file mode 100644
index 0000000..ce04973
--- /dev/null
+++ b/commons/wali/pom.xml
@@ -0,0 +1,42 @@
+<?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-parent</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+
+    <groupId>wali</groupId>
+    <artifactId>wali</artifactId>
+
+    <version>3.0.0-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-stream-utils</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+        </dependency>
+    </dependencies>
+</project>


[10/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..3aa2931
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponse.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..28615d0
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponseAction.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f17a66c
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpServer.java
@@ -0,0 +1,240 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..96943c2
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..4a69571
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..4d85d1a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..acd21e8
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+        }
+    }
+
+    @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/4d998c12/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
new file mode 100644
index 0000000..7c62d2f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..92a7d2a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListenerTest.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..2f16777
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/DelayedProtocolHandler.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..e80f52c
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/ReflexiveProtocolHandler.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..92eb78c
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/resources/logback-test.xml
@@ -0,0 +1,48 @@
+<?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/4d998c12/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
new file mode 100755
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100755
index 0000000..e8e4c2b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/ips.txt
@@ -0,0 +1,12 @@
+
+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/4d998c12/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
new file mode 100755
index 0000000..ea8c4bf
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/.gitignore
@@ -0,0 +1 @@
+/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..b163cd0
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/pom.xml
@@ -0,0 +1,60 @@
+<?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>remote-communications-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.apache.nifi</groupId>
+            <artifactId>nifi-utils</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/4d998c12/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
new file mode 100644
index 0000000..0092f7a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..5751c32
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..987ff65
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..848eb7e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 + "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..630631f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..57c1c30
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/protocol/DataFlow.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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();
+}


[31/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..acb47c5
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/docs/org.apache.nifi.processors.script.ExecuteScript/index.html
@@ -0,0 +1,264 @@
+<!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>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..3a34769
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteScript.java
@@ -0,0 +1,939 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..40f038d
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.js
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+with (Scripting) {
+    var instance = new ReaderScript({
+        validate: function () {
+            return ["This will never work."];
+        }
+    });
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..4b744ab
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.py
@@ -0,0 +1,19 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..076cdfa
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.rb
@@ -0,0 +1,21 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..d40623e
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.js
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..f4deb22
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.py
@@ -0,0 +1,22 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..0435ea7
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.rb
@@ -0,0 +1,30 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..827816c
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/Sub.py
@@ -0,0 +1,18 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..489036a
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.js
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..775aad4
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.rb
@@ -0,0 +1,17 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..39396ab
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.js
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..2429ea2
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.py
@@ -0,0 +1,26 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..f1b3ec5
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.rb
@@ -0,0 +1,29 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..d984dff
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/log4j.xml
@@ -0,0 +1,54 @@
+<?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/4d998c12/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
new file mode 100644
index 0000000..cf7c551
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.js
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..9f1a43d
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.py
@@ -0,0 +1,22 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..3fa53b4
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.rb
@@ -0,0 +1,39 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..c00537a
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.js
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..3495e4a
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.py
@@ -0,0 +1,26 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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/4d998c12/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
new file mode 100644
index 0000000..7c2eb23
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.rb
@@ -0,0 +1,31 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+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/4d998c12/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
new file mode 100644
index 0000000..901e158
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/parseXml.js
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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


[47/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..6d9cc91
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
@@ -0,0 +1,1183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ALL_ATTRIBUTES;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_DELINEATED_VALUES;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_MATCHING_ATTRIBUTES;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.AND;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_ATTRIBUTE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_DELINEATED_VALUE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_MATCHING_ATTRIBUTE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.APPEND;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ATTRIBUTE_REFERENCE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ATTR_NAME;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.CONTAINS;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.DIVIDE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ENDS_WITH;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EQUALS;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EQUALS_IGNORE_CASE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EXPRESSION;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FALSE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FIND;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FORMAT;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GREATER_THAN;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GREATER_THAN_OR_EQUAL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.HOSTNAME;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.INDEX_OF;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IP;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IS_NULL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LAST_INDEX_OF;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LENGTH;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LESS_THAN;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LESS_THAN_OR_EQUAL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MATCHES;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MINUS;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MOD;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MULTIPLY;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MULTI_ATTRIBUTE_REFERENCE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NEXT_INT;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOT;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOT_NULL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOW;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NUMBER;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.OR;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.PLUS;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.PREPEND;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_ALL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_NULL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.STARTS_WITH;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.STRING_LITERAL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_AFTER;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_AFTER_LAST;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_BEFORE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_BEFORE_LAST;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_DATE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_LOWER;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_NUMBER;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_RADIX;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_STRING;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_UPPER;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TRIM;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TRUE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.URL_DECODE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.URL_ENCODE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UUID;
+
+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.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.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.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;
+
+/**
+ * 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
+            buildEvaluator(tree);
+            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("$$", "$"));
+        }
+
+        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);
+    }
+
+    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);
+
+            return new Query(query, tree, buildEvaluator(tree));
+        } catch (final AttributeExpressionLanguageParsingException e) {
+            throw e;
+        } catch (final Exception e) {
+            throw new AttributeExpressionLanguageParsingException(e);
+        }
+    }
+
+    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");
+        }
+
+        Evaluator<?> chosenEvaluator = evaluator;
+        final Evaluator<?> rootEvaluator = getRootSubjectEvaluator(evaluator);
+        if (rootEvaluator != null) {
+            if (rootEvaluator instanceof MultiAttributeEvaluator) {
+                if (evaluator.getResultType() != ResultType.BOOLEAN) {
+                    throw new AttributeExpressionLanguageParsingException("Found Multi-Attribute function but return type is " + evaluator.getResultType() + ", not " + ResultType.BOOLEAN + ", for query: " + query);
+                }
+
+                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:
+                        chosenEvaluator = new AllAttributesEvaluator((BooleanEvaluator) evaluator, multiAttrEval);
+                        break;
+                }
+            }
+        }
+
+        return chosenEvaluator.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");
+        }
+        if (tree.getChildCount() == 1) {
+            return 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.
+            return buildFunctionExpressionEvaluator(tree, 0);
+        }
+    }
+
+    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);
+            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: {
+                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 IS_NULL: {
+                verifyArgCount(argEvaluators, 0, "isNull");
+                return new IsNullEvaluator(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/4d998c12/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
new file mode 100644
index 0000000..49ef6ef
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardAttributeExpression.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..cec73d1
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardExpressionLanguageCompiler.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..0affb7f
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..376ddfe
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanEvaluator.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..e5ef113
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanQueryResult.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..7474b60
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..a77bbe9
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateQueryResult.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..6d164df
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..403bae3
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..fc3c961
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberQueryResult.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..56bd76a
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/QueryResult.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1f4ff21
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringEvaluator.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+}


[09/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..3d5c75d
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Connectable.java
@@ -0,0 +1,291 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..0334bfb
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/ConnectableType.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..0a0089d
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Connection.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..cceca8f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Funnel.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..907dd92
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Port.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..75d04f5
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Position.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..cea13d2
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Size.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ef4b72a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java
@@ -0,0 +1,280 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..e1d2dd4
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java
@@ -0,0 +1,636 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..38df6f7
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Availability.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.controller;
+
+public enum Availability {
+
+    CLUSTER_MANAGER_ONLY,
+    NODE_ONLY,
+    BOTH;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..5b95524
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..eee878e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ContentAvailability.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..eaa0c48
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Counter.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..280f69d
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/EventBasedWorker.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1195bc9
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Heartbeater.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..303f540
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f6786fa
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..6b8ede0
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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();
+}


[48/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..500015f
--- /dev/null
+++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV2.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f937585
--- /dev/null
+++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV3.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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));
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..19f702c
--- /dev/null
+++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/Unpackage.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..24cd374
--- /dev/null
+++ b/commons/flowfile-packager/src/test/java/org/apache/nifi/util/TestPackageUnpackageV3.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/naive-search-ring-buffer/pom.xml
----------------------------------------------------------------------
diff --git a/commons/naive-search-ring-buffer/pom.xml b/commons/naive-search-ring-buffer/pom.xml
new file mode 100644
index 0000000..e84be0f
--- /dev/null
+++ b/commons/naive-search-ring-buffer/pom.xml
@@ -0,0 +1,30 @@
+<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-parent</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>naive-search-ring-buffer</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <packaging>jar</packaging>
+
+    <name>NiFi Ring Buffer</name>
+	
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/naive-search-ring-buffer/src/main/java/org/apache/nifi/util/NaiveSearchRingBuffer.java
----------------------------------------------------------------------
diff --git a/commons/naive-search-ring-buffer/src/main/java/org/apache/nifi/util/NaiveSearchRingBuffer.java b/commons/naive-search-ring-buffer/src/main/java/org/apache/nifi/util/NaiveSearchRingBuffer.java
new file mode 100644
index 0000000..85bfd96
--- /dev/null
+++ b/commons/naive-search-ring-buffer/src/main/java/org/apache/nifi/util/NaiveSearchRingBuffer.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/naive-search-ring-buffer/src/test/java/org/apache/nifi/util/TestNaiveSearchRingBuffer.java
----------------------------------------------------------------------
diff --git a/commons/naive-search-ring-buffer/src/test/java/org/apache/nifi/util/TestNaiveSearchRingBuffer.java b/commons/naive-search-ring-buffer/src/test/java/org/apache/nifi/util/TestNaiveSearchRingBuffer.java
new file mode 100644
index 0000000..0838e96
--- /dev/null
+++ b/commons/naive-search-ring-buffer/src/test/java/org/apache/nifi/util/TestNaiveSearchRingBuffer.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/commons/nifi-expression-language/pom.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-expression-language/pom.xml b/commons/nifi-expression-language/pom.xml
new file mode 100644
index 0000000..dfb1ea5
--- /dev/null
+++ b/commons/nifi-expression-language/pom.xml
@@ -0,0 +1,59 @@
+<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-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>
+                <version>3.5.2</version>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>antlr</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.antlr</groupId>
+            <artifactId>antlr-runtime</artifactId>
+            <version>3.5.2</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>[0.0.1-SNAPSHOT, 1.0.0-SNAPSHOT)</version>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..8cb6847
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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';
+NOT_NULL : 'notNull';
+TO_NUMBER : 'toNumber';
+URL_ENCODE : 'urlEncode';
+URL_DECODE : 'urlDecode';
+NOT : 'not';
+
+// 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';
+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';
+
+
+// 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/4d998c12/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
new file mode 100644
index 0000000..cf10fc0
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 | 
+				PREPEND | APPEND | FORMAT | STARTS_WITH | ENDS_WITH | CONTAINS) 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 | 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) 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/4d998c12/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
new file mode 100755
index 0000000..1b973c8
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/output/AttributeExpressionLexer.tokens
@@ -0,0 +1,72 @@
+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/4d998c12/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
new file mode 100644
index 0000000..81da47e
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..0d1b2c7
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+}


[25/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..dd3695c
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/SetUserAuthoritiesActionTest.java
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/impl/NiFiAuthorizationServiceTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/impl/NiFiAuthorizationServiceTest.java b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/impl/NiFiAuthorizationServiceTest.java
new file mode 100644
index 0000000..5c8b75a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/impl/NiFiAuthorizationServiceTest.java
@@ -0,0 +1,284 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 org.junit.Ignore;
+
+/**
+ *
+ */
+@Ignore
+public class NiFiAuthorizationServiceTest {
+
+//    private static final String UNKNOWN_USER_IN_CACHE_DN = "unknown-user-in-cache-dn";
+//    private static final String PENDING_USER_DN = "pending-user-dn";
+//    private static final String DISABLED_USER_DN = "disabled-user-dn";
+//    private static final String UNKNOWN_USER_IN_IDENTITY_PROVIDER_DN = "unknown-user-in-identity-provider-dn";
+//    private static final String ACCESS_EXCEPTION_IN_IDENTITY_PROVIDER_DN = "access-exception-in-identity-provider-dn";
+//    private static final String UNABLE_TO_UPDATE_CACHE_DN = "unable-to-update-cache-dn";
+//    private static final String VERIFICATION_REQUIRED_DN = "verification-required-dn";
+//    private static final String VERIFICATION_NOT_REQUIRED_DN = "verification-not-required-dn";
+//    private static final String NEW_USER_DN = "new-user-dn";
+//    
+//    private UserService userService;
+//    private AuthorityProvider authorityProvider;
+//    private UserDAO userDAO;
+//
+//    @Before
+//    public void setup() throws Exception {
+//        // mock the web security properties
+//        NiFiProperties properties = Mockito.mock(NiFiProperties.class);
+//        Mockito.when(properties.getSupportNewAccountRequests()).thenReturn(Boolean.TRUE);
+//        Mockito.when(properties.getUserCredentialCacheDurationSeconds()).thenReturn(60);
+//
+//        // mock the authority provider
+//
+//        // mock the admin service
+//        userDAO = Mockito.mock(UserDAO.class);
+//        Mockito.doAnswer(new Answer() {
+//
+//            @Override
+//            public Object answer(InvocationOnMock invocation) throws Throwable {
+//                Object[] args = invocation.getArguments();
+//                String dn = (String) args[0];
+//
+//                NiFiUser user = null;
+//                switch (dn) {
+//                    case PENDING_USER_DN:
+//                        user = new NiFiUser();
+//                        user.setDn(dn);
+//                        user.setStatus(AccountStatus.PENDING);
+//                        break;
+//                    case DISABLED_USER_DN:
+//                        user = new NiFiUser();
+//                        user.setDn(dn);
+//                        user.setStatus(AccountStatus.DISABLED);
+//                        break;
+//                    case UNKNOWN_USER_IN_IDENTITY_PROVIDER_DN:
+//                    case UNABLE_TO_UPDATE_CACHE_DN:
+//                    case ACCESS_EXCEPTION_IN_IDENTITY_PROVIDER_DN:
+//                        user = new NiFiUser();
+//                        user.setDn(dn);
+//                        user.setStatus(AccountStatus.ACTIVE);
+//                        break;
+//                    case VERIFICATION_REQUIRED_DN: {
+//                        Calendar calendar = Calendar.getInstance();
+//                        calendar.add(Calendar.SECOND, -65);
+//                        user = new NiFiUser();
+//                        user.setDn(dn);
+//                        user.setStatus(AccountStatus.ACTIVE);
+//                        user.setLastVerified(calendar.getTime());
+//                        user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_ADMIN, Authority.ROLE_DFM));
+//                        break;
+//                    }
+//                    case VERIFICATION_NOT_REQUIRED_DN: {
+//                        Calendar calendar = Calendar.getInstance();
+//                        calendar.add(Calendar.SECOND, -5);
+//                        user = new NiFiUser();
+//                        user.setDn(dn);
+//                        user.setStatus(AccountStatus.ACTIVE);
+//                        user.setLastVerified(calendar.getTime());
+//                        user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_ADMIN, Authority.ROLE_DFM));
+//                        break;
+//                    }
+//                }
+//                return user;
+//            }
+//        }).when(userDAO).getUser(Mockito.anyString());
+//        Mockito.doAnswer(new Answer() {
+//
+//            @Override
+//            public Object answer(InvocationOnMock invocation) throws Throwable {
+//                Object[] args = invocation.getArguments();
+//                NiFiUser user = (NiFiUser) args[0];
+//
+//                if (UNABLE_TO_UPDATE_CACHE_DN.equals(user.getDn())) {
+//                    throw new AdministrationException();
+//                }
+//                return user;
+//            }
+//        }).when(userDAO).updateUser(Mockito.any(NiFiUser.class));
+//        Mockito.doNothing().when(userDAO).createUser(Mockito.any(NiFiUser.class));
+//
+//        // mock the authority provider
+//        authorityProvider = Mockito.mock(AuthorityProvider.class);
+//        Mockito.doAnswer(new Answer() {
+//
+//            @Override
+//            public Object answer(InvocationOnMock invocation) throws Throwable {
+//                Object[] args = invocation.getArguments();
+//                String dn = (String) args[0];
+//
+//                boolean hasDn = false;
+//                if (VERIFICATION_REQUIRED_DN.equals(dn) || NEW_USER_DN.equals(dn)) {
+//                    hasDn = true;
+//                }
+//                return hasDn;
+//            }
+//        }).when(authorityProvider).doesDnExist(Mockito.anyString());
+//        Mockito.doAnswer(new Answer() {
+//
+//            @Override
+//            public Object answer(InvocationOnMock invocation) throws Throwable {
+//                Object[] args = invocation.getArguments();
+//                String dn = (String) args[0];
+//
+//                Set<String> authorities = null;
+//                switch (dn) {
+//                    case VERIFICATION_REQUIRED_DN:
+//                    case NEW_USER_DN:
+//                        authorities = new HashSet<>();
+//                        authorities.add("ROLE_MONITOR");
+//                        break;
+//                    case DISABLED_USER_DN:
+//                        throw new UnknownIdentityException("Unable to find user");
+//                }
+//                return authorities;
+//            }
+//        }).when(authorityProvider).getAuthorities(Mockito.anyString());
+//
+//        // create an instance of the authorization service
+//        userService = new UserServiceImpl();
+//        ((UserServiceImpl) userService).setAuthorityProvider(authorityProvider);
+//        ((UserServiceImpl) userService).set(authorityProvider);
+//        
+////        authorizationService.setIdentityProvider(identityProvider);
+////        authorizationService.setAuthorityProvider(authorityProvider);
+////        authorizationService.setProperties(properties);
+//    }
+//
+//    /**
+//     * Ensures the authorization service correctly handles users who are
+//     * unknown.
+//     *
+//     * @throws Exception
+//     */
+//    @Test(expected = org.springframework.security.core.userdetails.UsernameNotFoundException.class)
+//    public void testUnknownUserInCache() throws Exception {
+//        authorizationService.loadUserByUsername(WebUtils.formatProxyDn(UNKNOWN_USER_IN_CACHE_DN));
+//    }
+//
+//    /**
+//     * Ensures the authorization service correctly handles users whose accounts
+//     * are PENDING.
+//     *
+//     * @throws Exception
+//     */
+//    @Test(expected = nifi.admin.service.AccountPendingException.class)
+//    public void testPendingUser() throws Exception {
+//        authorizationService.loadUserByUsername(WebUtils.formatProxyDn(PENDING_USER_DN));
+//    }
+//
+//    /**
+//     * Ensures the authorization service correctly handles users whose accounts
+//     * are DISABLED.
+//     *
+//     * @throws Exception
+//     */
+//    @Test(expected = org.springframework.security.authentication.DisabledException.class)
+//    public void testDisabledUser() throws Exception {
+//        authorizationService.loadUserByUsername(WebUtils.formatProxyDn(DISABLED_USER_DN));
+//    }
+//
+//    /**
+//     * Ensures the authorization service correctly handles users whose are in
+//     * the cache but have been removed from the identity provider.
+//     *
+//     * @throws Exception
+//     */
+//    @Test(expected = org.springframework.security.authentication.DisabledException.class)
+//    public void testUnknownUserInIdentityProvider() throws Exception {
+//        authorizationService.loadUserByUsername(WebUtils.formatProxyDn(UNKNOWN_USER_IN_IDENTITY_PROVIDER_DN));
+//    }
+//
+//    /**
+//     * Ensures the authorization service correctly handles cases when the cache
+//     * is unable to be updated.
+//     *
+//     * @throws Exception
+//     */
+//    @Test(expected = org.springframework.security.authentication.AuthenticationServiceException.class)
+//    public void testUnableToUpdateCache() throws Exception {
+//        authorizationService.loadUserByUsername(WebUtils.formatProxyDn(UNABLE_TO_UPDATE_CACHE_DN));
+//    }
+//
+//    /**
+//     * Ensures the authorization service correctly handles cases when the
+//     * identity provider has an access exception.
+//     *
+//     * @throws Exception
+//     */
+//    @Test(expected = org.springframework.security.authentication.AuthenticationServiceException.class)
+//    public void testUnableToAccessIdentity() throws Exception {
+//        authorizationService.loadUserByUsername(WebUtils.formatProxyDn(ACCESS_EXCEPTION_IN_IDENTITY_PROVIDER_DN));
+//    }
+//
+//    /**
+//     * Ensures that user authorities are properly loaded from the authority
+//     * provider.
+//     *
+//     * @throws Exception
+//     */
+//    @Test
+//    public void testVerificationRequiredUser() throws Exception {
+//        NiFiUserDetails userDetails = (NiFiUserDetails) authorizationService.loadUserByUsername(WebUtils.formatProxyDn(VERIFICATION_REQUIRED_DN));
+//        NiFiUser user = userDetails.getNiFiUser();
+//        Mockito.verify(authorityProvider).getAuthorities(VERIFICATION_REQUIRED_DN);
+//
+//        // ensure the user details
+//        Assert.assertEquals(VERIFICATION_REQUIRED_DN, user.getDn());
+//        Assert.assertEquals(1, user.getAuthorities().size());
+//        Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_MONITOR));
+//    }
+//
+//    /**
+//     * Ensures that user authorities are not loaded when the cache is still
+//     * valid.
+//     *
+//     * @throws Exception
+//     */
+//    @Test
+//    public void testVerificationNotRequiredUser() throws Exception {
+//        NiFiUserDetails userDetails = (NiFiUserDetails) authorizationService.loadUserByUsername(WebUtils.formatProxyDn(VERIFICATION_NOT_REQUIRED_DN));
+//        NiFiUser user = userDetails.getNiFiUser();
+//        Mockito.verify(authorityProvider, Mockito.never()).getAuthorities(VERIFICATION_NOT_REQUIRED_DN);
+//
+//        // ensure the user details
+//        Assert.assertEquals(VERIFICATION_NOT_REQUIRED_DN, user.getDn());
+//        Assert.assertEquals(2, user.getAuthorities().size());
+//        Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_ADMIN));
+//        Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_DFM));
+//    }
+//
+//    /**
+//     * Ensures that new users are automatically created when the authority
+//     * provider has their authorities.
+//     *
+//     * @throws Exception
+//     */
+//    @Test
+//    public void testNewUser() throws Exception {
+//        NiFiUserDetails userDetails = (NiFiUserDetails) authorizationService.loadUserByUsername(WebUtils.formatProxyDn(NEW_USER_DN));
+//        NiFiUser user = userDetails.getNiFiUser();
+//        Mockito.verify(authorityProvider).getAuthorities(NEW_USER_DN);
+//
+//        // ensure the user details
+//        Assert.assertEquals(NEW_USER_DN, user.getDn());
+//        Assert.assertEquals(1, user.getAuthorities().size());
+//        Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_MONITOR));
+//    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100755
index 0000000..cd1a4e7
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/.gitignore
@@ -0,0 +1,6 @@
+/target
+/target
+/target
+/target
+/target
+/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..4b210da
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/pom.xml
@@ -0,0 +1,47 @@
+<?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>
+                        <phase>verify</phase>
+                        <goals>
+                            <goal>jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..0e2dcb0
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/AboutDTO.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..70c408b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BannerDTO.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ddc3d2e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinBoardDTO.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..c6aca24
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinDTO.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..015b174
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinQueryDTO.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..53100e3
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ClusterDTO.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1be480c
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectableDTO.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..660820c
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectionDTO.java
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b916025
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..9e15fc1
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerDTO.java
@@ -0,0 +1,262 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..10ea41d
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/CounterDTO.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ac1aa38
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/CountersDTO.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..5a2d789
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..61c3c33
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1240501
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/FunnelDTO.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f50c792
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/LabelDTO.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..e3c8445
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+}


[27/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..89661b2
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAuthoritiesCacheAction.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..288e297
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserCacheAction.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..56b214c
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserGroupAction.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..127f1df
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java
@@ -0,0 +1,230 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..63aa93b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardUserService.java
@@ -0,0 +1,629 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.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.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 id
+     */
+    @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 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/4d998c12/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
new file mode 100644
index 0000000..edd214b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/Transaction.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..2d2ef82
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionBuilder.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..924e01f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionException.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..a3cfb5e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransaction.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b6e5a30
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransactionBuilder.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..b05d32f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/AuthorityProviderFactoryBean.java
@@ -0,0 +1,516 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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 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 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/4d998c12/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
new file mode 100644
index 0000000..0535e27
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderConfigurationContext.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..e4b16c4
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderInitializationContext.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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;
+    }
+
+}


[44/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..d7569e0
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/NumberLiteralEvaluator.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..d739ac7
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/StringLiteralEvaluator.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..d9dd4d3
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..9192958
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..8c07278
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyMatchingAttributeEvaluator.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..209c86f
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/DelineatedAttributeEvaluator.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f80ed97
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiAttributeEvaluator.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..9a441ce
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiMatchAttributeEvaluator.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..6dabc0a
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..47d42cb
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/AttributeExpressionLanguageException.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f8531cb
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/AttributeExpressionLanguageParsingException.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..4a9a9c5
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/IllegalAttributeException.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 {
+
+    public IllegalAttributeException() {
+        super();
+    }
+
+    public IllegalAttributeException(final String explanation) {
+        super(explanation);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..a2b7214
--- /dev/null
+++ b/commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
@@ -0,0 +1,1068 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.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");
+    }
+
+    @Test
+    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");
+        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 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).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()}"));
+    }
+    
+    
+    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/4d998c12/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
new file mode 100644
index 0000000..398a23b
--- /dev/null
+++ b/commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/nifi-file-utils/pom.xml
----------------------------------------------------------------------
diff --git a/commons/nifi-file-utils/pom.xml b/commons/nifi-file-utils/pom.xml
new file mode 100644
index 0000000..e3cf792
--- /dev/null
+++ b/commons/nifi-file-utils/pom.xml
@@ -0,0 +1,35 @@
+<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-parent</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-file-utils</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <packaging>jar</packaging>
+    <name>NiFi File Utils</name>
+
+    <dependencies>
+        <dependency>
+            <groupId>commons-codec</groupId>
+            <artifactId>commons-codec</artifactId>
+            <version>1.10</version>
+        </dependency>
+    </dependencies>
+</project>


[15/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..4e3b932
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/ReflexiveProtocolHandler.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100755
index 0000000..ea8c4bf
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-web/.gitignore
@@ -0,0 +1 @@
+/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..a7c39c6
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-web/pom.xml
@@ -0,0 +1,50 @@
+<?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/4d998c12/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
new file mode 100644
index 0000000..44fb25a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContext.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..06907d2
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextImpl.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..012e7c7
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextThreadLocal.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..90b8a37
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100755
index 0000000..ea8c4bf
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/.gitignore
@@ -0,0 +1 @@
+/target

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..ad5dda7
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/pom.xml
@@ -0,0 +1,133 @@
+<?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-file-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>
+            <version>1.9</version>
+        </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/4d998c12/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
new file mode 100644
index 0000000..183c7ca
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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");
+                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/4d998c12/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
new file mode 100644
index 0000000..6bc5d6c
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/Event.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f9dfb00
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/EventManager.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..7fadc78
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/impl/EventManagerImpl.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..2e3d278
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/firewall/ClusterNodeFirewall.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..bcee661
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewall.java
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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/4d998c12/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
new file mode 100644
index 0000000..eedb88f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..6ff15a7
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..a273704
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowDao.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..339d904
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b3afc6e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ce5a08b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/StaleFlowException.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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() {
+    }
+
+}


[03/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..c67181a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowFromDOMFactory.java
@@ -0,0 +1,418 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f1ee760
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSerializationException.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..331b26c
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSerializer.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..706ac46
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSynchronizationException.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..f6889fe
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowSynchronizer.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..fa33b49
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowUnmarshaller.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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/4d998c12/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
new file mode 100644
index 0000000..415472f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/SnippetManager.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.io.ByteArrayInputStream;
+import org.apache.nifi.io.ByteArrayOutputStream;
+import org.apache.nifi.io.DataOutputStream;
+import org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..2899a85
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardCounter.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..e08a94d
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowSerializer.java
@@ -0,0 +1,404 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+    }
+
+}


[22/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..d6ea8a0
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceResultsDTO.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..af89f97
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceSearchableFieldDTO.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..745105e
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageDTO.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..23cf892
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageRequestDTO.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..77b6e7a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageResultsDTO.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f3bbcef
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceLinkDTO.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..c15c598
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceNodeDTO.java
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..83d7a91
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/ComponentSearchResultDTO.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..7c250ee
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/NodeSearchResultDTO.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..212d4f2
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/SearchResultsDTO.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ab5636d
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/UserGroupSearchResultDTO.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..9402eeb
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/UserSearchResultDTO.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f0e11d6
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterConnectionStatusDTO.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..4d7fa4d
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterPortStatusDTO.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..b9f45f2
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterProcessorStatusDTO.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..d6b26d4
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterRemoteProcessGroupStatusDTO.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..d833991
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusDTO.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..31a9948
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusHistoryDTO.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..387cce5
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusDTO.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..9e063e4
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.java
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..f7cba44
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusDTO.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..93d89ac
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodePortStatusDTO.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..afff4b1
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusDTO.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..03923bf
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusDTO.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..03647cd
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusDTO.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..c8b05f3
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusHistoryDTO.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+
+}


[36/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..95cf4da
--- /dev/null
+++ b/commons/wali/src/main/java/org/wali/MinimalLockingWriteAheadLog.java
@@ -0,0 +1,1008 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR 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.io.BufferedInputStream;
+import org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..bbc7efb
--- /dev/null
+++ b/commons/wali/src/main/java/org/wali/SerDe.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR 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/4d998c12/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
new file mode 100644
index 0000000..ffb11ca
--- /dev/null
+++ b/commons/wali/src/main/java/org/wali/SyncListener.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR 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/4d998c12/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
new file mode 100644
index 0000000..1b039f8
--- /dev/null
+++ b/commons/wali/src/main/java/org/wali/UpdateType.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.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/4d998c12/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
new file mode 100644
index 0000000..4567872
--- /dev/null
+++ b/commons/wali/src/main/java/org/wali/WriteAheadRepository.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR 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/4d998c12/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
new file mode 100644
index 0000000..e0f7f96
--- /dev/null
+++ b/commons/wali/src/test/java/org/wali/DummyRecord.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR 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/4d998c12/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
new file mode 100644
index 0000000..8cc7860
--- /dev/null
+++ b/commons/wali/src/test/java/org/wali/DummyRecordSerde.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR 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/4d998c12/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
new file mode 100644
index 0000000..57f3495
--- /dev/null
+++ b/commons/wali/src/test/java/org/wali/TestMinimalLockingWriteAheadLog.java
@@ -0,0 +1,298 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR 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/4d998c12/extensions/file-authorization-provider/pom.xml
----------------------------------------------------------------------
diff --git a/extensions/file-authorization-provider/pom.xml b/extensions/file-authorization-provider/pom.xml
new file mode 100644
index 0000000..f8d823f
--- /dev/null
+++ b/extensions/file-authorization-provider/pom.xml
@@ -0,0 +1,90 @@
+<?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-parent</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>file-authorization-provider</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <name>Authorization Provider: File</name>
+
+    <build>
+        <resources>
+            <resource>
+                <directory>src/main/resources</directory>
+            </resource>
+            <resource>
+                <directory>src/main/xsd</directory>
+            </resource>
+        </resources>
+        <plugins>
+            <plugin>
+                <groupId>com.sun.tools.xjc.maven2</groupId>
+                <artifactId>maven-jaxb-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>current</id>
+                        <goals>
+                            <goal>generate</goal>
+                        </goals>
+                        <configuration>
+                            <schemaDirectory>src/main/xsd</schemaDirectory>
+                            <includeSchemas>
+                                <includeSchema>**/*.xsd</includeSchema>
+                            </includeSchemas>
+                            <generatePackage>org.apache.nifi.user.generated</generatePackage>
+                        </configuration>
+                    </execution>
+                </executions>
+                <configuration>
+                    <generateDirectory>${project.build.directory}/generated-sources/jaxb</generateDirectory>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>[0.0.1-SNAPSHOT, 1.0.0-SNAPSHOT)</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-file-utils</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-properties</artifactId>
+            <version>[0.0.1-SNAPSHOT, 1.0.0-SNAPSHOT)</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-lang</groupId>
+            <artifactId>commons-lang</artifactId>
+            <version>2.6</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-codec</groupId>
+            <artifactId>commons-codec</artifactId>
+            <version>1.10</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+</project>


[21/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..c32efdb
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusDTO.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..4fa2b64
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.java
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..334adad
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessorStatusDTO.java
@@ -0,0 +1,203 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..a8fcc9f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemotePortStatusDTO.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..363d4da
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.java
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..e4e7a45
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDTO.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..a203e10
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDescriptorDTO.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..5b97c16
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDTO.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b9d16bb
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDetailDTO.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..cc6894a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusSnapshotDTO.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..f860b30
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/DateTimeAdapter.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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);
+        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);
+        parser.setTimeZone(TimeZone.getDefault());
+        return parser.parse(date);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..ab77370
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/TimeAdapter.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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);
+        formatter.setTimeZone(TimeZone.getDefault());
+        return formatter.format(date);
+    }
+
+    @Override
+    public Date unmarshal(String date) throws Exception {
+        final SimpleDateFormat parser = new SimpleDateFormat(DEFAULT_TIME_FORMAT);
+        parser.setTimeZone(TimeZone.getDefault());
+        return parser.parse(date);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..49b90a2
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/TimestampAdapter.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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);
+        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);
+        parser.setTimeZone(TimeZone.getDefault());
+        return parser.parse(date);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..36fc163
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/AboutEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1b2c0b6
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ActionEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..eed1305
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/AuthorityEntity.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..594707a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/BannerEntity.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..4287893
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/BulletinBoardEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..732a4b7
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterConnectionStatusEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..8306b0f
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..16a2497
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterPortStatusEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..68e5c5c
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessorStatusEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..8a3da7c
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterRemoteProcessGroupStatusEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..546bd8b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterSearchResultsEntity.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..d22d432
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..9ed69fd
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusHistoryEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..69ae937
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..c69c69a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionsEntity.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..e13903b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerConfigurationEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..62d0409
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..94d8aca
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerStatusEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+
+}


[33/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapPutResult.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapPutResult.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapPutResult.java
new file mode 100644
index 0000000..29695eb
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapPutResult.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server.map;
+
+import java.nio.ByteBuffer;
+
+public class MapPutResult {
+    private final boolean successful;
+    private final ByteBuffer key, value;
+    private final ByteBuffer existingValue;
+    private final ByteBuffer evictedKey, evictedValue;
+    
+    public MapPutResult(final boolean successful, final ByteBuffer key, final ByteBuffer value, final ByteBuffer existingValue, final ByteBuffer evictedKey, final ByteBuffer evictedValue) {
+        this.successful = successful;
+        this.key = key;
+        this.value = value;
+        this.existingValue = existingValue;
+        this.evictedKey = evictedKey;
+        this.evictedValue = evictedValue;
+    }
+
+    public boolean isSuccessful() {
+        return successful;
+    }
+
+    public ByteBuffer getKey() {
+        return key;
+    }
+
+    public ByteBuffer getValue() {
+        return value;
+    }
+    
+    public ByteBuffer getExistingValue() {
+        return existingValue;
+    }
+
+    public ByteBuffer getEvictedKey() {
+        return evictedKey;
+    }
+
+    public ByteBuffer getEvictedValue() {
+        return evictedValue;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/PersistentMapCache.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/PersistentMapCache.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/PersistentMapCache.java
new file mode 100644
index 0000000..77fb77d
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/PersistentMapCache.java
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server.map;
+
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.wali.MinimalLockingWriteAheadLog;
+import org.wali.SerDe;
+import org.wali.UpdateType;
+import org.wali.WriteAheadRepository;
+
+public class PersistentMapCache implements MapCache {
+
+    private final MapCache wrapped;
+    private final WriteAheadRepository<MapWaliRecord> wali;
+    
+    private final AtomicLong modifications = new AtomicLong(0L);
+    
+    public PersistentMapCache(final String serviceIdentifier, final File persistencePath, final MapCache cacheToWrap) throws IOException {
+        wali = new MinimalLockingWriteAheadLog<>(persistencePath.toPath(), 1, new Serde(), null);
+        wrapped = cacheToWrap;
+    }
+
+    synchronized void restore() throws IOException {
+        final Collection<MapWaliRecord> recovered = wali.recoverRecords();
+        for ( final MapWaliRecord record : recovered ) {
+            if ( record.getUpdateType() == UpdateType.CREATE ) {
+                wrapped.putIfAbsent(record.getKey(), record.getValue());
+            }
+        }
+    }
+
+    @Override
+    public MapPutResult putIfAbsent(final ByteBuffer key, final ByteBuffer value) throws IOException {
+        final MapPutResult putResult = wrapped.putIfAbsent(key, value);
+        if ( putResult.isSuccessful() ) {
+            // The put was successful.
+            final MapWaliRecord record = new MapWaliRecord(UpdateType.CREATE, key, value);
+            final List<MapWaliRecord> records = new ArrayList<>();
+            records.add(record);
+
+            if ( putResult.getEvictedKey() != null ) {
+                records.add(new MapWaliRecord(UpdateType.DELETE, putResult.getEvictedKey(), putResult.getEvictedValue()));
+            }
+            
+            wali.update(Collections.singletonList(record), false);
+            
+            final long modCount = modifications.getAndIncrement();
+            if ( modCount > 0 && modCount % 100000 == 0 ) {
+                wali.checkpoint();
+            }
+        }
+        
+        return putResult;
+    }
+
+    @Override
+    public boolean containsKey(final ByteBuffer key) throws IOException {
+        return wrapped.containsKey(key);
+    }
+
+    @Override
+    public ByteBuffer get(final ByteBuffer key) throws IOException {
+        return wrapped.get(key);
+    }
+
+    @Override
+    public ByteBuffer remove(ByteBuffer key) throws IOException {
+        final ByteBuffer removeResult = wrapped.remove(key);
+        if ( removeResult != null ) {
+            final MapWaliRecord record = new MapWaliRecord(UpdateType.DELETE, key, removeResult);
+            final List<MapWaliRecord> records = new ArrayList<>(1);
+            records.add(record);
+            wali.update(records, false);
+            
+            final long modCount = modifications.getAndIncrement();
+            if ( modCount > 0 && modCount % 1000 == 0 ) {
+                wali.checkpoint();
+            }
+        }
+        return removeResult;
+    }
+
+
+    @Override
+    public void shutdown() throws IOException {
+        wali.shutdown();
+    }
+
+
+    private static class MapWaliRecord {
+        private final UpdateType updateType;
+        private final ByteBuffer key;
+        private final ByteBuffer value;
+        
+        public MapWaliRecord(final UpdateType updateType, final ByteBuffer key, final ByteBuffer value) {
+            this.updateType = updateType;
+            this.key = key;
+            this.value = value;
+        }
+        
+        public UpdateType getUpdateType() {
+            return updateType;
+        }
+        
+        public ByteBuffer getKey() {
+            return key;
+        }
+        
+        public ByteBuffer getValue() {
+            return value;
+        }
+    }
+    
+    private static class Serde implements SerDe<MapWaliRecord> {
+
+        @Override
+        public void serializeEdit(MapWaliRecord previousRecordState, MapWaliRecord newRecordState, java.io.DataOutputStream out) throws IOException {
+            final UpdateType updateType = newRecordState.getUpdateType();
+            if ( updateType == UpdateType.DELETE ) {
+                out.write(0);
+            } else {
+                out.write(1);
+            }
+            
+            final byte[] key = newRecordState.getKey().array();
+            final byte[] value = newRecordState.getValue().array();
+            
+            out.writeInt(key.length);
+            out.write(key);
+            out.writeInt(value.length);
+            out.write(value);
+        }
+
+        @Override
+        public void serializeRecord(MapWaliRecord record, java.io.DataOutputStream out) throws IOException {
+            serializeEdit(null, record, out);
+        }
+
+        @Override
+        public MapWaliRecord deserializeEdit(final DataInputStream in, final Map<Object, MapWaliRecord> currentRecordStates, final int version) throws IOException {
+            final int updateTypeValue = in.read();
+            if ( updateTypeValue < 0 ) {
+                throw new EOFException();
+            }
+
+            final UpdateType updateType = (updateTypeValue == 0 ? UpdateType.DELETE : UpdateType.CREATE);
+            
+            final int keySize = in.readInt();
+            final byte[] key = new byte[keySize];
+            in.readFully(key);
+
+            final int valueSize = in.readInt();
+            final byte[] value = new byte[valueSize];
+            in.readFully(value);
+
+            return new MapWaliRecord(updateType, ByteBuffer.wrap(key), ByteBuffer.wrap(value));
+        }
+
+        @Override
+        public MapWaliRecord deserializeRecord(DataInputStream in, int version) throws IOException {
+            return deserializeEdit(in, new HashMap<Object, MapWaliRecord>(), version);
+        }
+
+        @Override
+        public Object getRecordIdentifier(final MapWaliRecord record) {
+            return record.getKey();
+        }
+
+        @Override
+        public UpdateType getUpdateType(final MapWaliRecord record) {
+            return record.getUpdateType();
+        }
+
+        @Override
+        public String getLocation(final MapWaliRecord record) {
+            return null;
+        }
+
+        @Override
+        public int getVersion() {
+            return 1;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/SimpleMapCache.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/SimpleMapCache.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/SimpleMapCache.java
new file mode 100644
index 0000000..10139f1
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/SimpleMapCache.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server.map;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.nifi.distributed.cache.server.EvictionPolicy;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SimpleMapCache implements MapCache {
+    private static final Logger logger = LoggerFactory.getLogger(SimpleMapCache.class);
+
+    private final Map<ByteBuffer, MapCacheRecord> cache = new HashMap<>();
+    private final SortedMap<MapCacheRecord, ByteBuffer> inverseCacheMap;
+    
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Lock readLock = rwLock.readLock();
+    private final Lock writeLock = rwLock.writeLock();
+    
+    private final String serviceIdentifier;
+    
+    private final int maxSize;
+    
+    public SimpleMapCache(final String serviceIdentifier, final int maxSize, final EvictionPolicy evictionPolicy) {
+        // need to change to ConcurrentMap as this is modified when only the readLock is held
+        inverseCacheMap = new ConcurrentSkipListMap<>(evictionPolicy.getComparator());
+        this.serviceIdentifier = serviceIdentifier;
+        this.maxSize = maxSize;
+    }
+    
+    @Override
+    public String toString() {
+        return "SimpleSetCache[service id=" + serviceIdentifier + "]";
+    }
+
+    // don't need synchronized because this method is only called when the writeLock is held, and all 
+    // public methods obtain either the read or write lock
+    private MapCacheRecord evict() {
+        if ( cache.size() < maxSize ) {
+            return null;
+        }
+        
+        final MapCacheRecord recordToEvict = inverseCacheMap.firstKey();
+        final ByteBuffer valueToEvict = inverseCacheMap.remove(recordToEvict);
+        cache.remove(valueToEvict);
+        
+        if ( logger.isDebugEnabled() ) {
+            logger.debug("Evicting value {} from cache", new String(valueToEvict.array(), StandardCharsets.UTF_8));
+        }
+        
+        return recordToEvict;
+    }
+
+    @Override
+    public MapPutResult putIfAbsent(final ByteBuffer key, final ByteBuffer value) {
+        writeLock.lock();
+        try {
+            final MapCacheRecord record = cache.get(key);
+            if ( record == null ) {
+                // Record is null. We will add.
+                final MapCacheRecord evicted = evict();
+                final MapCacheRecord newRecord = new MapCacheRecord(key, value);
+                cache.put(key, newRecord);
+                inverseCacheMap.put(newRecord, key);
+                
+                if ( evicted == null ) {
+                    return new MapPutResult(true, key, value, null, null, null);
+                } else {
+                    return new MapPutResult(true, key, value, null, evicted.getKey(), evicted.getValue());
+                }
+            }
+            
+            // Record is not null. Increment hit count and return result indicating that record was not added.
+            inverseCacheMap.remove(record);
+            record.hit();
+            inverseCacheMap.put(record, key);
+            
+            return new MapPutResult(false, key, value, record.getValue(), null, null);
+        } finally {
+            writeLock.unlock();
+        }
+    }
+    
+    @Override
+    public boolean containsKey(final ByteBuffer key) {
+        readLock.lock();
+        try {
+            final MapCacheRecord record = cache.get(key);
+            if ( record == null ) {
+                return false;
+            }
+            
+            inverseCacheMap.remove(record);
+            record.hit();
+            inverseCacheMap.put(record, key);
+            
+            return true;
+        } finally {
+            readLock.unlock();
+        }
+    }
+
+    @Override
+    public ByteBuffer get(final ByteBuffer key) {
+        readLock.lock();
+        try {
+            final MapCacheRecord record = cache.get(key);
+            if ( record == null ) {
+                return null;
+            }
+            
+            inverseCacheMap.remove(record);
+            record.hit();
+            inverseCacheMap.put(record, key);
+            
+            return record.getValue();
+        } finally {
+            readLock.unlock();
+        }
+    }
+
+    @Override
+    public ByteBuffer remove(ByteBuffer key) throws IOException {
+        writeLock.lock();
+        try {
+            final MapCacheRecord record = cache.remove(key);
+            if (record == null) {
+                return null;
+            }
+            inverseCacheMap.remove(record);
+            return record.getValue();
+        } finally {
+            writeLock.unlock();
+        }
+    }
+
+    @Override
+    public void shutdown() throws IOException {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/PersistentSetCache.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/PersistentSetCache.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/PersistentSetCache.java
new file mode 100644
index 0000000..4d75fc0
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/PersistentSetCache.java
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server.set;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.wali.MinimalLockingWriteAheadLog;
+import org.wali.SerDe;
+import org.wali.UpdateType;
+import org.wali.WriteAheadRepository;
+
+public class PersistentSetCache implements SetCache {
+
+    private final SetCache wrapped;
+    private final WriteAheadRepository<SetRecord> wali;
+    
+    private final AtomicLong modifications = new AtomicLong(0L);
+    
+    public PersistentSetCache(final String serviceIdentifier, final File persistencePath, final SetCache cacheToWrap) throws IOException {
+        wali = new MinimalLockingWriteAheadLog<>(persistencePath.toPath(), 1, new Serde(), null);
+        wrapped = cacheToWrap;
+    }
+    
+    public synchronized void restore() throws IOException {
+        final Collection<SetRecord> recovered = wali.recoverRecords();
+        for ( final SetRecord record : recovered ) {
+            if ( record.getUpdateType() == UpdateType.CREATE ) {
+                addIfAbsent(record.getBuffer());
+            }
+        }
+    }
+    
+    @Override
+    public synchronized SetCacheResult remove(final ByteBuffer value) throws IOException {
+        final SetCacheResult removeResult = wrapped.remove(value);
+        if ( removeResult.getResult() ) {
+            final SetRecord record = new SetRecord(UpdateType.DELETE, value);
+            final List<SetRecord> records = new ArrayList<>();
+            records.add(record);
+            wali.update(records, false);
+            
+            final long modCount = modifications.getAndIncrement();
+            if ( modCount > 0 && modCount % 1000 == 0 ) {
+                wali.checkpoint();
+            }
+        }
+
+        return removeResult;
+    }
+
+    @Override
+    public synchronized SetCacheResult addIfAbsent(final ByteBuffer value) throws IOException {
+        final SetCacheResult addResult = wrapped.addIfAbsent(value);
+        if ( addResult.getResult() ) {
+            final SetRecord record = new SetRecord(UpdateType.CREATE, value);
+            final List<SetRecord> records = new ArrayList<>();
+            records.add(record);
+            
+            final SetCacheRecord evictedRecord = addResult.getEvictedRecord();
+            if ( evictedRecord != null ) {
+                records.add(new SetRecord(UpdateType.DELETE, evictedRecord.getValue()));
+            }
+            
+            wali.update(records, false);
+            
+            final long modCount = modifications.getAndIncrement();
+            if ( modCount > 0 && modCount % 1000 == 0 ) {
+                wali.checkpoint();
+            }
+        }
+        
+        return addResult;
+    }
+
+    @Override
+    public synchronized SetCacheResult contains(final ByteBuffer value) throws IOException {
+        return wrapped.contains(value);
+    }
+    
+    @Override
+    public void shutdown() throws IOException {
+        wali.shutdown();
+    }
+    
+    private static class SetRecord {
+        private final UpdateType updateType;
+        private final ByteBuffer value;
+        
+        public SetRecord(final UpdateType updateType, final ByteBuffer value) {
+            this.updateType = updateType;
+            this.value = value;
+        }
+        
+        public UpdateType getUpdateType() {
+            return updateType;
+        }
+        
+        public ByteBuffer getBuffer() {
+            return value;
+        }
+        
+        public byte[] getData() {
+            return value.array();
+        }
+    }
+    
+    private static class Serde implements SerDe<SetRecord> {
+
+        @Override
+        public void serializeEdit(final SetRecord previousRecordState, final SetRecord newRecordState, final DataOutputStream out) throws IOException {
+            final UpdateType updateType = newRecordState.getUpdateType();
+            if ( updateType == UpdateType.DELETE ) {
+                out.write(0);
+            } else {
+                out.write(1);
+            }
+            
+            final byte[] data = newRecordState.getData();
+            out.writeInt(data.length);
+            out.write(newRecordState.getData());
+        }
+
+        @Override
+        public void serializeRecord(SetRecord record, DataOutputStream out) throws IOException {
+            serializeEdit(null, record, out);
+        }
+
+        @Override
+        public SetRecord deserializeEdit(final DataInputStream in, final Map<Object, SetRecord> currentRecordStates, final int version) throws IOException {
+            final int value = in.read();
+            if ( value < 0 ) {
+                throw new EOFException();
+            }
+
+            final UpdateType updateType = (value == 0 ? UpdateType.DELETE : UpdateType.CREATE);
+            
+            final int size = in.readInt();
+            final byte[] data = new byte[size];
+            in.readFully(data);
+            
+            return new SetRecord(updateType, ByteBuffer.wrap(data));
+        }
+
+        @Override
+        public SetRecord deserializeRecord(DataInputStream in, int version) throws IOException {
+            return deserializeEdit(in, new HashMap<Object, SetRecord>(), version);
+        }
+
+        @Override
+        public Object getRecordIdentifier(final SetRecord record) {
+            return record.getBuffer();
+        }
+
+        @Override
+        public UpdateType getUpdateType(final SetRecord record) {
+            return record.getUpdateType();
+        }
+
+        @Override
+        public String getLocation(final SetRecord record) {
+            return null;
+        }
+
+        @Override
+        public int getVersion() {
+            return 1;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCache.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCache.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCache.java
new file mode 100644
index 0000000..bf6ae3e
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCache.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server.set;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public interface SetCache {
+
+    SetCacheResult remove(ByteBuffer value) throws IOException;
+    SetCacheResult addIfAbsent(ByteBuffer value) throws IOException;
+    SetCacheResult contains(ByteBuffer value) throws IOException;
+    void shutdown() throws IOException;
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCacheRecord.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCacheRecord.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCacheRecord.java
new file mode 100644
index 0000000..20b6fae
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCacheRecord.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server.set;
+
+import java.nio.ByteBuffer;
+
+import org.apache.nifi.distributed.cache.server.CacheRecord;
+
+public class SetCacheRecord extends CacheRecord {
+    private final ByteBuffer value;
+    
+    public SetCacheRecord(final ByteBuffer value) {
+        this.value = value;
+    }
+    
+    public ByteBuffer getValue() {
+        return value;
+    }
+    
+    @Override
+    public int hashCode() {
+        return value.hashCode();
+    }
+    
+    @Override
+    public boolean equals(final Object obj) {
+        if ( this == obj ) {
+            return true;
+        }
+        
+        if (obj instanceof SetCacheRecord) {
+            return value.equals(((SetCacheRecord) obj).value);
+        }
+        return false;
+    }
+    
+    @Override
+    public String toString() {
+        return "SetCacheRecord[value=" + new String(value.array()) + ", hitCount=" + getHitCount() + "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCacheResult.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCacheResult.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCacheResult.java
new file mode 100644
index 0000000..732c4f0
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCacheResult.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server.set;
+
+
+
+public class SetCacheResult {
+    private final boolean result;
+    private final SetCacheRecord stats;
+    private final SetCacheRecord evictedRecord;
+    
+    public SetCacheResult(final boolean result, final SetCacheRecord stats, final SetCacheRecord evictedRecord) {
+        this.result = result;
+        this.stats = stats;
+        this.evictedRecord = evictedRecord;
+    }
+    
+    public boolean getResult() {
+        return result;
+    }
+    
+    public SetCacheRecord getRecord() {
+        return stats;
+    }
+    
+    public SetCacheRecord getEvictedRecord() {
+        return evictedRecord;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SimpleSetCache.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SimpleSetCache.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SimpleSetCache.java
new file mode 100644
index 0000000..77d6481
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SimpleSetCache.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server.set;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.nifi.distributed.cache.server.EvictionPolicy;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SimpleSetCache implements SetCache {
+    private static final Logger logger = LoggerFactory.getLogger(SimpleSetCache.class);
+    
+    private final Map<ByteBuffer, SetCacheRecord> cache = new HashMap<>();
+    private final SortedMap<SetCacheRecord, ByteBuffer> inverseCacheMap;
+    
+    private final String serviceIdentifier;
+    
+    private final int maxSize;
+    
+    public SimpleSetCache(final String serviceIdentifier, final int maxSize, final EvictionPolicy evictionPolicy) {
+        inverseCacheMap = new TreeMap<>(evictionPolicy.getComparator());
+        this.serviceIdentifier = serviceIdentifier;
+        this.maxSize = maxSize;
+    }
+    
+    private synchronized SetCacheRecord evict() {
+        if ( cache.size() < maxSize ) {
+            return null;
+        }
+        
+        final SetCacheRecord recordToEvict = inverseCacheMap.firstKey();
+        final ByteBuffer valueToEvict = inverseCacheMap.remove(recordToEvict);
+        cache.remove(valueToEvict);
+        
+        if ( logger.isDebugEnabled() ) {
+            logger.debug("Evicting value {} from cache", new String(valueToEvict.array(), StandardCharsets.UTF_8));
+        }
+        
+        return recordToEvict;
+    }
+    
+    @Override
+    public synchronized SetCacheResult addIfAbsent(final ByteBuffer value) {
+        final SetCacheRecord record = cache.get(value);
+        if ( record == null ) {
+            final SetCacheRecord evicted = evict();
+            final SetCacheRecord newRecord = new SetCacheRecord(value);
+            cache.put(value, newRecord);
+            inverseCacheMap.put(newRecord, value);
+            return new SetCacheResult(true, newRecord, evicted);
+        } else {
+            // We have to remove the record and add it again in order to cause the Map to stay sorted
+            inverseCacheMap.remove(record);
+            record.hit();
+            inverseCacheMap.put(record, value);
+            
+            return new SetCacheResult(false, record, null);
+        }
+    }
+    
+    @Override
+    public synchronized SetCacheResult contains(final ByteBuffer value) {
+        final SetCacheRecord record = cache.get(value);
+        if ( record == null ) {
+            return new SetCacheResult(false, null, null);
+        } else {
+            // We have to remove the record and add it again in order to cause the Map to stay sorted
+            inverseCacheMap.remove(record);
+            record.hit();
+            inverseCacheMap.put(record, value);
+            
+            return new SetCacheResult(true, record, null);
+        }
+    }
+    
+    @Override
+    public synchronized SetCacheResult remove(final ByteBuffer value) {
+        final SetCacheRecord record = cache.remove(value);
+        if ( record == null ) {
+            return new SetCacheResult(false, null, null);
+        } else {
+            inverseCacheMap.remove(record);
+            return new SetCacheResult(true, record, null);
+        }
+    }
+    
+    @Override
+    public String toString() {
+        return "SimpleSetCache[service id=" + serviceIdentifier + "]";
+    }
+    
+    @Override
+    public void shutdown() throws IOException {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
new file mode 100644
index 0000000..0509c7c
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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.distributed.cache.server.DistributedSetCacheServer
+org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/index.html
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/index.html b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/index.html
new file mode 100644
index 0000000..dca3aa1
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/index.html
@@ -0,0 +1,82 @@
+<!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>Distributed Map Cache Client Service</title>
+<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
+</head>
+
+<body>
+	<h2>Description:</h2>
+
+	<p>A Controller Service that starts an embedded server and listens for connections from clients. The
+	server provides the ability to query the cache, add data to the cache, and remove data from the cache.</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.</p>
+
+	<ul>
+		<li><strong>Port</strong>
+			<ul>
+				<li>The port to listen on for incoming connections</li>
+				<li>Default value: 4557</li>
+				<li>Supports expression language: false</li>
+			</ul></li>
+		<li>SSL Context Service
+			<ul>
+				<li>If specified, this service will be used to create an SSL Context that will be used to secure communications; if not specified, communications will not be secure</li>
+				<li>Default value: no default</li>
+				<li>Supports expression language: false</li>
+			</ul></li>
+		<li><strong>Maximum Cache Entries</strong>
+			<ul>
+				<li>The maximum number of cache entries that the cache can hold
+				<li>Default value: 10,000</li>
+				<li>Supports expression language: false</li>
+			</ul></li>
+		<li><strong>Eviction Strategy</strong>
+			<ul>
+				<li>Determines which strategy should be used to evict values from the cache to make room for new entries. Valid values: 
+					<code>Least Frequently Used</code>, <code>Least Recently Used</code>, and <code>First In, First Out</code>
+				<li>Default value: Least Frequently Used</li>
+				<li>Supports expression language: false</li>
+			</ul></li>
+		<li>Persistence Directory
+			<ul>
+				<li>If specified, the cache will be persisted in the given directory; if not specified, the cache will be in-memory only</li>
+				<li>Default value: no default (in-memory)</li>
+				<li>Supports expression language: true - JVM and System Properties Only</li>
+			</ul></li>
+	</ul>
+
+
+	<i>See Also:</i>
+	<ul>
+		<li><a href="../org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/index.html">Distributed Map Cache Client Service</a></li>
+		<li><a href="../org.apache.nifi.ssl.StandardSSLContextService/index.html">Standard SSL Context Service</a></li>
+	</ul>
+
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/java/org/apache/nifi/distributed/cache/server/TestServerAndClient.java
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/java/org/apache/nifi/distributed/cache/server/TestServerAndClient.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/java/org/apache/nifi/distributed/cache/server/TestServerAndClient.java
new file mode 100644
index 0000000..b5f3fd6
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/java/org/apache/nifi/distributed/cache/server/TestServerAndClient.java
@@ -0,0 +1,530 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.distributed.cache.server;
+
+import org.apache.nifi.distributed.cache.server.DistributedSetCacheServer;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.ConnectException;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.distributed.cache.client.Deserializer;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService;
+import org.apache.nifi.distributed.cache.client.DistributedSetCacheClientService;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.distributed.cache.client.exception.DeserializationException;
+import org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.ssl.SSLContextService.ClientAuth;
+import org.apache.nifi.ssl.StandardSSLContextService;
+import org.apache.nifi.util.MockConfigurationContext;
+import org.apache.nifi.util.MockControllerServiceInitializationContext;
+
+import org.apache.commons.lang3.SerializationException;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestServerAndClient {
+
+    private static Logger LOGGER;
+
+    static {
+        System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
+        System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.distributed.cache.server.AbstractCacheServer", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.distributed.cache.client.DistributedMapCacheClientService", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.distributed.cache.server.TestServerAndClient", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.remote.io.socket.ssl.SSLSocketChannel", "trace");
+        LOGGER = LoggerFactory.getLogger(TestServerAndClient.class);
+    }
+
+    @Test
+    public void testNonPersistentSetServerAndClient() throws InitializationException, IOException {
+        LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
+        // Create server
+        final DistributedSetCacheServer server = new DistributedSetCacheServer();
+        MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
+        server.initialize(serverInitContext);
+
+        final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
+        final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
+        server.startServer(serverContext);
+
+        final DistributedSetCacheClientService client = createClient();
+        final Serializer<String> serializer = new StringSerializer();
+        final boolean added = client.addIfAbsent("test", serializer);
+        assertTrue(added);
+
+        final boolean contains = client.contains("test", serializer);
+        assertTrue(contains);
+
+        final boolean addedAgain = client.addIfAbsent("test", serializer);
+        assertFalse(addedAgain);
+
+        final boolean removed = client.remove("test", serializer);
+        assertTrue(removed);
+
+        final boolean containedAfterRemove = client.contains("test", serializer);
+        assertFalse(containedAfterRemove);
+
+        server.shutdownServer();
+    }
+
+    @Test
+    public void testPersistentSetServerAndClient() throws InitializationException, IOException {
+        LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
+        // Create server
+        final DistributedSetCacheServer server = new DistributedSetCacheServer();
+        MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
+        server.initialize(serverInitContext);
+
+        final File dataFile = new File("target/cache-data");
+        deleteRecursively(dataFile);
+
+        final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
+        serverProperties.put(DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath());
+        final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
+        server.startServer(serverContext);
+
+        final DistributedSetCacheClientService client = createClient();
+        final Serializer<String> serializer = new StringSerializer();
+        final boolean added = client.addIfAbsent("test", serializer);
+        final boolean added2 = client.addIfAbsent("test2", serializer);
+        assertTrue(added);
+        assertTrue(added2);
+
+        final boolean contains = client.contains("test", serializer);
+        final boolean contains2 = client.contains("test2", serializer);
+        assertTrue(contains);
+        assertTrue(contains2);
+
+        final boolean addedAgain = client.addIfAbsent("test", serializer);
+        assertFalse(addedAgain);
+
+        final boolean removed = client.remove("test", serializer);
+        assertTrue(removed);
+
+        final boolean containedAfterRemove = client.contains("test", serializer);
+        assertFalse(containedAfterRemove);
+
+        server.shutdownServer();
+
+        final DistributedSetCacheServer newServer = new DistributedSetCacheServer();
+        MockControllerServiceInitializationContext newServerInitContext = new MockControllerServiceInitializationContext(newServer, "server2");
+        newServer.initialize(newServerInitContext);
+
+        final MockConfigurationContext newServerContext = new MockConfigurationContext(serverProperties,
+                newServerInitContext.getControllerServiceLookup());
+        newServer.startServer(newServerContext);
+
+        assertFalse(client.contains("test", serializer));
+        assertTrue(client.contains("test2", serializer));
+
+        newServer.shutdownServer();
+    }
+
+    @Test
+    public void testPersistentSetServerAndClientWithLFUEvictions() throws InitializationException, IOException {
+        LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
+        // Create server
+        final DistributedSetCacheServer server = new DistributedSetCacheServer();
+        MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
+        server.initialize(serverInitContext);
+
+        final File dataFile = new File("target/cache-data");
+        deleteRecursively(dataFile);
+
+        final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
+        serverProperties.put(DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath());
+        serverProperties.put(DistributedSetCacheServer.MAX_CACHE_ENTRIES, "3");
+
+        final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
+        server.startServer(serverContext);
+
+        final DistributedSetCacheClientService client = createClient();
+        final Serializer<String> serializer = new StringSerializer();
+        final boolean added = client.addIfAbsent("test", serializer);
+        waitABit();
+        final boolean added2 = client.addIfAbsent("test2", serializer);
+        waitABit();
+        final boolean added3 = client.addIfAbsent("test3", serializer);
+        waitABit();
+        assertTrue(added);
+        assertTrue(added2);
+        assertTrue(added3);
+
+        final boolean contains = client.contains("test", serializer);
+        final boolean contains2 = client.contains("test2", serializer);
+        assertTrue(contains);
+        assertTrue(contains2);
+
+        final boolean addedAgain = client.addIfAbsent("test", serializer);
+        assertFalse(addedAgain);
+
+        final boolean added4 = client.addIfAbsent("test4", serializer);
+        assertTrue(added4);
+
+        // ensure that added3 was evicted because it was used least frequently
+        assertFalse(client.contains("test3", serializer));
+
+        server.shutdownServer();
+
+        final DistributedSetCacheServer newServer = new DistributedSetCacheServer();
+        MockControllerServiceInitializationContext newServerInitContext = new MockControllerServiceInitializationContext(newServer, "server2");
+        newServer.initialize(newServerInitContext);
+
+        final MockConfigurationContext newServerContext = new MockConfigurationContext(serverProperties,
+                newServerInitContext.getControllerServiceLookup());
+        newServer.startServer(newServerContext);
+
+        assertTrue(client.contains("test", serializer));
+        assertTrue(client.contains("test2", serializer));
+        assertFalse(client.contains("test3", serializer));
+        assertTrue(client.contains("test4", serializer));
+
+        newServer.shutdownServer();
+    }
+
+    @Test
+    public void testPersistentSetServerAndClientWithFIFOEvictions() throws InitializationException, IOException {
+        LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
+        // Create server
+        final DistributedSetCacheServer server = new DistributedSetCacheServer();
+        MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
+        server.initialize(serverInitContext);
+
+        final File dataFile = new File("target/cache-data");
+        deleteRecursively(dataFile);
+
+        final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
+        serverProperties.put(DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath());
+        serverProperties.put(DistributedSetCacheServer.MAX_CACHE_ENTRIES, "3");
+        serverProperties.put(DistributedSetCacheServer.EVICTION_POLICY, DistributedSetCacheServer.EVICTION_STRATEGY_FIFO);
+
+        final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
+        server.startServer(serverContext);
+
+        final DistributedSetCacheClientService client = createClient();
+        final Serializer<String> serializer = new StringSerializer();
+
+        // add 3 entries to the cache. But, if we add too fast, we'll have the same millisecond
+        // for the entry time so we don't know which entry will be evicted. So we wait a few millis in between
+        final boolean added = client.addIfAbsent("test", serializer);
+        waitABit();
+        final boolean added2 = client.addIfAbsent("test2", serializer);
+        waitABit();
+        final boolean added3 = client.addIfAbsent("test3", serializer);
+        waitABit();
+
+        assertTrue(added);
+        assertTrue(added2);
+        assertTrue(added3);
+
+        final boolean contains = client.contains("test", serializer);
+        final boolean contains2 = client.contains("test2", serializer);
+        assertTrue(contains);
+        assertTrue(contains2);
+
+        final boolean addedAgain = client.addIfAbsent("test", serializer);
+        assertFalse(addedAgain);
+
+        final boolean added4 = client.addIfAbsent("test4", serializer);
+        assertTrue(added4);
+
+        // ensure that added3 was evicted because it was used least frequently
+        assertFalse(client.contains("test", serializer));
+        assertTrue(client.contains("test3", serializer));
+
+        server.shutdownServer();
+
+        final DistributedSetCacheServer newServer = new DistributedSetCacheServer();
+        MockControllerServiceInitializationContext newServerInitContext = new MockControllerServiceInitializationContext(newServer, "server2");
+        newServer.initialize(newServerInitContext);
+
+        final MockConfigurationContext newServerContext = new MockConfigurationContext(serverProperties,
+                newServerInitContext.getControllerServiceLookup());
+        newServer.startServer(newServerContext);
+
+        assertFalse(client.contains("test", serializer));
+        assertTrue(client.contains("test2", serializer));
+        assertTrue(client.contains("test3", serializer));
+        assertTrue(client.contains("test4", serializer));
+
+        newServer.shutdownServer();
+    }
+
+    @Test
+    public void testNonPersistentMapServerAndClient() throws InitializationException, IOException, InterruptedException {
+        LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
+        // Create server
+        final DistributedMapCacheServer server = new DistributedMapCacheServer();
+        MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
+        server.initialize(serverInitContext);
+
+        final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
+        final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
+        server.startServer(serverContext);
+
+        DistributedMapCacheClientService client = new DistributedMapCacheClientService();
+        MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client");
+        client.initialize(clientInitContext);
+
+        final Map<PropertyDescriptor, String> clientProperties = new HashMap<>();
+        clientProperties.put(DistributedMapCacheClientService.HOSTNAME, "localhost");
+        clientProperties.put(DistributedMapCacheClientService.COMMUNICATIONS_TIMEOUT, "360 secs");
+        MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup());
+        client.cacheConfig(clientContext);
+        final Serializer<String> valueSerializer = new StringSerializer();
+        final Serializer<String> keySerializer = new StringSerializer();
+        final Deserializer<String> deserializer = new StringDeserializer();
+
+        final String original = client.getAndPutIfAbsent("testKey", "test", keySerializer, valueSerializer, deserializer);
+        assertEquals(null, original);
+        LOGGER.debug("end getAndPutIfAbsent");
+
+        final boolean contains = client.containsKey("testKey", keySerializer);
+        assertTrue(contains);
+        LOGGER.debug("end containsKey");
+
+        final boolean added = client.putIfAbsent("testKey", "test", keySerializer, valueSerializer);
+        assertFalse(added);
+        LOGGER.debug("end putIfAbsent");
+
+        final String originalAfterPut = client.getAndPutIfAbsent("testKey", "test", keySerializer, valueSerializer, deserializer);
+        assertEquals("test", originalAfterPut);
+        LOGGER.debug("end getAndPutIfAbsent");
+
+        final boolean removed = client.remove("testKey", keySerializer);
+        assertTrue(removed);
+        LOGGER.debug("end remove");
+
+        final boolean containedAfterRemove = client.containsKey("testKey", keySerializer);
+        assertFalse(containedAfterRemove);
+
+        client.putIfAbsent("testKey", "test", keySerializer, valueSerializer);
+        client.close();
+        try {
+            client.containsKey("testKey", keySerializer);
+            fail("Should be closed and not accessible");
+        } catch (Exception e) {
+
+        }
+        client = null;
+        clientInitContext = null;
+        clientContext = null;
+
+        DistributedMapCacheClientService client2 = new DistributedMapCacheClientService();
+
+        MockControllerServiceInitializationContext clientInitContext2 = new MockControllerServiceInitializationContext(client2, "client2");
+        client2.initialize(clientInitContext2);
+
+        MockConfigurationContext clientContext2 = new MockConfigurationContext(clientProperties,
+                clientInitContext2.getControllerServiceLookup());
+        client2.cacheConfig(clientContext2);
+        assertFalse(client2.putIfAbsent("testKey", "test", keySerializer, valueSerializer));
+        assertTrue(client2.containsKey("testKey", keySerializer));
+        server.shutdownServer();
+        Thread.sleep(1000);
+        try {
+            client2.containsKey("testKey", keySerializer);
+            fail("Should have blown exception!");
+        } catch (ConnectException e) {
+            client2 = null;
+            clientContext2 = null;
+            clientInitContext2 = null;
+        }
+        Thread.sleep(2000);
+        System.gc();
+        LOGGER.debug("end testNonPersistentMapServerAndClient");
+    }
+
+    @Test
+    public void testClientTermination() throws InitializationException, IOException, InterruptedException {
+        LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
+        // Create server
+        final DistributedMapCacheServer server = new DistributedMapCacheServer();
+        MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
+        server.initialize(serverInitContext);
+
+        final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
+        final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
+        server.startServer(serverContext);
+
+        DistributedMapCacheClientService client = new DistributedMapCacheClientService();
+        MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client");
+        client.initialize(clientInitContext);
+
+        final Map<PropertyDescriptor, String> clientProperties = new HashMap<>();
+        clientProperties.put(DistributedMapCacheClientService.HOSTNAME, "localhost");
+        clientProperties.put(DistributedMapCacheClientService.COMMUNICATIONS_TIMEOUT, "360 secs");
+        MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup());
+        client.cacheConfig(clientContext);
+        final Serializer<String> valueSerializer = new StringSerializer();
+        final Serializer<String> keySerializer = new StringSerializer();
+        final Deserializer<String> deserializer = new StringDeserializer();
+
+        final String original = client.getAndPutIfAbsent("testKey", "test", keySerializer, valueSerializer, deserializer);
+        assertEquals(null, original);
+
+        final boolean contains = client.containsKey("testKey", keySerializer);
+        assertTrue(contains);
+
+        final boolean added = client.putIfAbsent("testKey", "test", keySerializer, valueSerializer);
+        assertFalse(added);
+
+        final String originalAfterPut = client.getAndPutIfAbsent("testKey", "test", keySerializer, valueSerializer, deserializer);
+        assertEquals("test", originalAfterPut);
+
+        final boolean removed = client.remove("testKey", keySerializer);
+        assertTrue(removed);
+
+        final boolean containedAfterRemove = client.containsKey("testKey", keySerializer);
+        assertFalse(containedAfterRemove);
+
+        client = null;
+        clientInitContext = null;
+        clientContext = null;
+        Thread.sleep(2000);
+        System.gc();
+        server.shutdownServer();
+    }
+
+    @Ignore
+    @Test
+    public void testSSLWith2RequestsWithServerTimeout() throws InitializationException, IOException, InterruptedException {
+        LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
+        // Create SSLContext Service
+        final StandardSSLContextService sslService = new StandardSSLContextService();
+        final MockControllerServiceInitializationContext sslServerInitContext = new MockControllerServiceInitializationContext(sslService,
+                "ssl-context");
+        sslService.initialize(sslServerInitContext);
+
+        final Map<PropertyDescriptor, String> sslServerProps = new HashMap<>();
+        sslServerProps.put(StandardSSLContextService.KEYSTORE, "src/test/resources/localhost-ks.jks");
+        sslServerProps.put(StandardSSLContextService.KEYSTORE_PASSWORD, "localtest");
+        sslServerProps.put(StandardSSLContextService.KEYSTORE_TYPE, "JKS");
+        sslServerProps.put(StandardSSLContextService.TRUSTSTORE, "src/test/resources/localhost-ts.jks");
+        sslServerProps.put(StandardSSLContextService.TRUSTSTORE_PASSWORD, "localtest");
+        sslServerProps.put(StandardSSLContextService.TRUSTSTORE_TYPE, "JKS");
+        MockConfigurationContext sslServerContext = new MockConfigurationContext(sslServerProps, sslServerInitContext);
+        sslService.onConfigured(sslServerContext);
+        sslService.createSSLContext(ClientAuth.REQUIRED);
+        // Create server
+        final DistributedMapCacheServer server = new DistributedMapCacheServer();
+        final MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
+        server.initialize(serverInitContext);
+
+        final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
+        serverProperties.put(DistributedMapCacheServer.SSL_CONTEXT_SERVICE, "ssl-context");
+        final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
+        server.startServer(serverContext);
+
+        DistributedMapCacheClientService client = new DistributedMapCacheClientService();
+        MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client");
+        client.initialize(clientInitContext);
+
+        final Map<PropertyDescriptor, String> clientProperties = new HashMap<>();
+        clientProperties.put(DistributedMapCacheClientService.HOSTNAME, "localhost");
+        clientProperties.put(DistributedMapCacheClientService.COMMUNICATIONS_TIMEOUT, "360 secs");
+        clientProperties.put(DistributedMapCacheClientService.SSL_CONTEXT_SERVICE, "ssl-context");
+        MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup());
+        client.cacheConfig(clientContext);
+        final Serializer<String> valueSerializer = new StringSerializer();
+        final Serializer<String> keySerializer = new StringSerializer();
+        final Deserializer<String> deserializer = new StringDeserializer();
+
+        final String original = client.getAndPutIfAbsent("testKey", "test", keySerializer, valueSerializer, deserializer);
+        assertEquals(null, original);
+
+        Thread.sleep(30000);
+        try {
+            final boolean contains = client.containsKey("testKey", keySerializer);
+            assertTrue(contains);
+        } catch (IOException e) {
+            // this is due to the server timing out in the middle of this request
+            assertTrue(e.getMessage().contains("Channel is closed"));
+        }
+
+        server.shutdownServer();
+    }
+
+    private void waitABit() {
+        try {
+            Thread.sleep(10L);
+        } catch (final InterruptedException e) {
+        }
+    }
+
+    private DistributedSetCacheClientService createClient() throws InitializationException {
+        final DistributedSetCacheClientService client = new DistributedSetCacheClientService();
+        MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client");
+        client.initialize(clientInitContext);
+
+        final Map<PropertyDescriptor, String> clientProperties = new HashMap<>();
+        clientProperties.put(DistributedSetCacheClientService.HOSTNAME, "localhost");
+        final MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup());
+        client.onConfigured(clientContext);
+
+        return client;
+    }
+
+    private static class StringSerializer implements Serializer<String> {
+        @Override
+        public void serialize(final String value, final OutputStream output) throws SerializationException, IOException {
+            output.write(value.getBytes(StandardCharsets.UTF_8));
+        }
+    }
+
+    private static class StringDeserializer implements Deserializer<String> {
+        @Override
+        public String deserialize(final byte[] input) throws DeserializationException, IOException {
+            return (input.length == 0) ? null : new String(input, StandardCharsets.UTF_8);
+        }
+    }
+
+    private static void deleteRecursively(final File dataFile) throws IOException {
+        if (dataFile == null || !dataFile.exists()) {
+            return;
+        }
+
+        final File[] children = dataFile.listFiles();
+        for (final File child : children) {
+            if (child.isDirectory()) {
+                deleteRecursively(child);
+            } else {
+                for (int i = 0; i < 100 && child.exists(); i++) {
+                    child.delete();
+                }
+
+                if (child.exists()) {
+                    throw new IOException("Could not delete " + dataFile.getAbsolutePath());
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ks.jks
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ks.jks b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ks.jks
new file mode 100755
index 0000000..81be31d
Binary files /dev/null and b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ks.jks differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ts.jks
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ts.jks b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ts.jks
new file mode 100755
index 0000000..820e1e1
Binary files /dev/null and b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ts.jks differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/distributed-cache-services-nar/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-services-nar/pom.xml b/nar-bundles/distributed-cache-services-bundle/distributed-cache-services-nar/pom.xml
new file mode 100644
index 0000000..75cab34
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-services-nar/pom.xml
@@ -0,0 +1,49 @@
+<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>distributed-cache-services-bundle</artifactId>
+		<version>0.0.1-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>distributed-cache-services-nar</artifactId>
+	<name>Distributed Cache Services NAR</name>
+	<packaging>nar</packaging>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>standard-services-api-nar</artifactId>
+			<type>nar</type>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>distributed-cache-client-service</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>distributed-cache-protocol</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>distributed-cache-server</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+	</dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/distributed-cache-services-bundle/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/distributed-cache-services-bundle/pom.xml b/nar-bundles/distributed-cache-services-bundle/pom.xml
new file mode 100644
index 0000000..dcfa541
--- /dev/null
+++ b/nar-bundles/distributed-cache-services-bundle/pom.xml
@@ -0,0 +1,83 @@
+<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>standard-services-api-bundle</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>distributed-cache-services-bundle</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <name>Distributed Cache Services Bundle</name>
+    <packaging>pom</packaging>
+
+    <modules>
+        <module>distributed-cache-protocol</module>
+        <module>distributed-cache-client-service</module>
+        <module>distributed-cache-server</module>
+        <module>distributed-cache-services-nar</module>
+    </modules>
+
+
+    <dependencyManagement>
+        <dependencies>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>distributed-cache-client-service-api</artifactId>
+                <version>${standard.services.api.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>ssl-context-service-api</artifactId>
+                <version>${standard.services.api.version}</version>
+            </dependency>
+            <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>remote-communications-utils</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>wali</groupId>
+                <artifactId>wali</artifactId>
+                <version>3.0.0-SNAPSHOT</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>ssl-context-service</artifactId>
+                <version>0.0.1-SNAPSHOT</version>
+                <scope>test</scope>
+            </dependency>
+        </dependencies>
+    </dependencyManagement>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..a5d3d11
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/pom.xml
@@ -0,0 +1,81 @@
+<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>
+
+


[04/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..a0a07f2
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -0,0 +1,3534 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.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(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);
+                }
+            });
+
+            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 (Boolean.TRUE.equals(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 if (remoteInputSocketPort == null) {
+            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);
+        }
+    }
+
+    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);
+            }
+
+            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.</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 {
+        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));
+
+        // TODO: We should only call this the first time that it is added to the graph....
+        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();
+            }
+        } 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.getSentCount();
+                bytesSent += remoteStatus.getSentContentSize();
+            }
+        }
+
+        // connection status
+        final Collection<ConnectionStatus> connectionStatusCollection = new ArrayList<>();
+        status.setConnectionStatus(connectionStatusCollection);
+
+        // get the connection and remote port status
+        for (final Connection conn : group.getConnections()) {
+            final ConnectionStatus connStatus = new ConnectionStatus();
+            connStatus.setId(conn.getIdentifier());
+            connStatus.setGroupId(conn.getProcessGroup().getIdentifier());
+            connStatus.setSourceId(conn.getSource().getIdentifier());
+            connStatus.setSourceName(conn.getSource().getName());
+            connStatus.setDestinationId(conn.getDestination().getIdentifier());
+            connStatus.setDestinationName(conn.getDestination().getName());
+
+            final FlowFileEvent connectionStatusReport = statusReport.getReportEntry(conn.getIdentifier());
+            if (connectionStatusReport != null) {
+                connStatus.setInputBytes(connectionStatusReport.getContentSizeIn());
+                connStatus.setInputCount(connectionStatusReport.getFlowFilesIn());
+                connStatus.setOutputBytes(connectionStatusReport.getContentSizeOut());
+                connStatus.setOutputCount(connectionStatusReport.getFlowFilesOut());
+            }
+
+            if (StringUtils.isNotBlank(conn.getName())) {
+                connStatus.setName(conn.getName());
+            } else if (conn.getRelationships() != null && !conn.getRelationships().isEmpty()) {
+                final Collection<String> relationships = new ArrayList<>(conn.getRelationships().size());
+                for (final Relationship relationship : conn.getRelationships()) {
+                    relationships.add(relationship.getName());
+                }
+         

<TRUNCATED>

[49/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..afb56e8
--- /dev/null
+++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..cfbae88
--- /dev/null
+++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java
@@ -0,0 +1,752 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..9a9a27d
--- /dev/null
+++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..0aaf5ef
--- /dev/null
+++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EdgeNode.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..12d9a4f
--- /dev/null
+++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..c36c38d
--- /dev/null
+++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileLineage.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..fdc7470
--- /dev/null
+++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileNode.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/commons/flowfile-packager/pom.xml
----------------------------------------------------------------------
diff --git a/commons/flowfile-packager/pom.xml b/commons/flowfile-packager/pom.xml
new file mode 100644
index 0000000..f4001fe
--- /dev/null
+++ b/commons/flowfile-packager/pom.xml
@@ -0,0 +1,43 @@
+<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-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>
+            <version>1.9</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+            <version>3.3.2</version>
+        </dependency>
+    </dependencies>
+
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..ae16f99
--- /dev/null
+++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackager.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..2437279
--- /dev/null
+++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV1.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.escapeXml(entry.getKey());
+            final String escapedValue = StringEscapeUtils.escapeXml(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/4d998c12/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
new file mode 100644
index 0000000..6f9d6b1
--- /dev/null
+++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV2.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..181f3e3
--- /dev/null
+++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV3.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..fd9d92d
--- /dev/null
+++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackager.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..f8ef3d1
--- /dev/null
+++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV1.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+        }
+    }
+}


[23/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..0b20852
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO.java
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..156a6e9
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/TemplateDTO.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..85c82dc
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/UserDTO.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..285c355
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/UserGroupDTO.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..8402480
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/ActionDTO.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..9ab47ed
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/HistoryDTO.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..e8e11e5
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/HistoryQueryDTO.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..58086ce
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..3523f62
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..2809398
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/RemoteProcessGroupDetailsDTO.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..4074cba
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ActionDetailsDTO.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..4011b00
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ConfigureDetailsDTO.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ba88bd1
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ConnectDetailsDTO.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..c6f0450
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/MoveDetailsDTO.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b5a5bcb
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/PurgeDetailsDTO.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ed58077
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/AttributeDTO.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..90dd119
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceDTO.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b1ead42
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceEventDTO.java
@@ -0,0 +1,630 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..7f939b9
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceOptionsDTO.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..e9e5453
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceRequestDTO.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+}


[05/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..3af2098
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java
@@ -0,0 +1,709 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.io.BufferedOutputStream;
+import org.apache.nifi.processor.QueueSize;
+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");
+    public static final int SWAP_ENCODING_VERSION = 6;
+
+    private final ScheduledExecutorService swapQueueIdentifierExecutor;
+    private final ScheduledExecutorService swapInExecutor;
+    private volatile FlowFileRepository flowFileRepository;
+
+    // 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() {
+        this.storageDirectory = NiFiProperties.getInstance().getSwapStorageLocation();
+        if (!storageDirectory.exists() && !storageDirectory.mkdirs()) {
+            throw new RuntimeException("Cannot create Swap Storage directory " + storageDirectory.getAbsolutePath());
+        }
+
+        swapQueueIdentifierExecutor = new FlowEngine(1, "Identifies Queues for FlowFile Swapping");
+
+        final NiFiProperties properties = NiFiProperties.getInstance();
+        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) {
+        this.claimManager = claimManager;
+        this.flowFileRepository = flowFileRepository;
+        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()) {
+                                    logger.warn("Swapped in FlowFiles from file " + swapFile.getAbsolutePath() + " but failed to delete the file; this file can be cleaned up manually");
+                                }
+                            } catch (final Exception e) {
+                                logger.error("Failed to Swap In FlowFiles for {} due to {}", new Object[]{flowFileQueue, e.toString()}, e);
+                                if (swapFile != null) {
+                                    queue.add(swapFile);
+                                }
+                            }
+                        }
+                    } finally {
+                        queueLockWrapper.getLock().unlock();
+                    }
+                }
+            }
+        }
+    }
+
+    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 String swapLocation = swapFile.getAbsolutePath();
+                    final List<FlowFileRecord> toSwap = flowFileQueue.pollSwappableRecords();
+
+                    int recordsSwapped;
+                    try (final FileOutputStream fos = new FileOutputStream(swapFile)) {
+                        recordsSwapped = serializeFlowFiles(toSwap, flowFileQueue, swapLocation, fos);
+                        flowFileRepository.swapFlowFilesOut(toSwap, flowFileQueue, swapLocation);
+                        fos.getFD().sync();
+                    } catch (final IOException ioe) {
+                        recordsSwapped = 0;
+                        flowFileQueue.putSwappedRecords(toSwap);
+                        logger.error("Failed to swap out {} FlowFiles from {} to Swap File {} due to {}", new Object[]{toSwap.size(), flowFileQueue, swapLocation, ioe.toString()}, 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 {
+                        swapFile.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();
+            }
+        });
+
+        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) {
+            // 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) {
+                    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();
+                final FlowFileQueue queue = queueMap.get(connectionId);
+                if (queue == null) {
+                    logger.error("Cannot recover Swapped FlowFiles from Swap File {} because the FlowFiles belong to a Connection with ID {} and that Connection does not exist", swapFile, connectionId);
+                    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) {
+                logger.error("Cannot recover Swapped FlowFiles from Swap File {} due to {}", swapFile, ioe.toString());
+                if (logger.isDebugEnabled()) {
+                    logger.error("", 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;
+        }
+    }
+}


[46/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..c010bb1
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringQueryResult.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..a88b72b
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/BooleanCastEvaluator.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b6d758c
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/DateCastEvaluator.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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);
+
+                    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 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);
+
+                        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/4d998c12/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
new file mode 100644
index 0000000..0447d48
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/NumberCastEvaluator.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1a3732a
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/StringCastEvaluator.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..9ed8f06
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AndEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..bd5e81a
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AppendEvaluator.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1b4b74e
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..99802f6
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ContainsEvaluator.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..0594c4a
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DateToNumberEvaluator.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..c7583cb
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DivideEvaluator.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..c31c2b1
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EndsWithEvaluator.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ec67b0f
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsEvaluator.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..344395b
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsIgnoreCaseEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..b377c4a
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ccb1a89
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FormatEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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).format(subjectValue));
+    }
+
+    @Override
+    public Evaluator<?> getSubjectEvaluator() {
+        return subject;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..6c712bb
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..98951f2
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanOrEqualEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..c0b1134
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/HostnameEvaluator.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..21f9fe8
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IPEvaluator.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..80422cb
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IndexOfEvaluator.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..a9d678c
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsNullEvaluator.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..2b67093
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LastIndexOfEvaluator.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..dec1f8f
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LengthEvaluator.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..9a58910
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..10f3f6a
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanOrEqualEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1bb86a5
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..0665979
--- /dev/null
+++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MinusEvaluator.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+    }
+
+}


[32/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..9058cf4
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/processors/script/ExecuteScript.java
@@ -0,0 +1,566 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..7be47a8
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ConverterScript.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..883b688
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JRubyScriptFactory.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..774fb1f
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JavaScriptScriptFactory.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..6b40b5e
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JythonScriptFactory.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..d879722
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/OutputStreamHandler.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.scripting;
+
+import java.io.OutputStream;
+
+public interface OutputStreamHandler {
+
+    void write(OutputStream out);
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..b1d89c0
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ReaderScript.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..786f541
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/Script.java
@@ -0,0 +1,303 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..6f38886
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ScriptEngineFactory.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..da18606
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ScriptFactory.java
@@ -0,0 +1,269 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..7eef98b
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/WriterScript.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..20a3982
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -0,0 +1,15 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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


[11/51] [partial] incubator-nifi git commit: Initial code contribution

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..84565da
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/node/Node.java
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..e26d196
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..ef72298
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..7169730
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..1ed5b30
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..68c29bc
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/main/resources/nifi-cluster-manager-context.xml
@@ -0,0 +1,124 @@
+<?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/4d998c12/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
new file mode 100644
index 0000000..09ea44b
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..2fcf7ef
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewallTest.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 org.apache.nifi.cluster.firewall.impl.FileBasedClusterNodeFirewall;
+import java.io.File;
+import java.io.IOException;
+import org.apache.nifi.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/4d998c12/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
new file mode 100644
index 0000000..6294dfc
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java
@@ -0,0 +1,341 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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");
+        restoreLocation = new File(System.getProperty("java.io.tmpdir") + "/restore");
+
+        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);
+            }
+        }
+
+    }
+
+    @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/4d998c12/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
new file mode 100644
index 0000000..0c65aba
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java
@@ -0,0 +1,368 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..d45a4d1
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/4d998c12/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
new file mode 100644
index 0000000..7347a94
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/TestWebClusterManager.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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 org.junit.Test;
+
+public class TestWebClusterManager {
+
+    @Test
+    public void testNormalizedStatusSnapshotDate() throws ParseException {
+        final DateFormat df = new SimpleDateFormat("yyyy/MM/dd HH:mm:SS.SSS");
+        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);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/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
new file mode 100644
index 0000000..35380dd
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpRequest.java
@@ -0,0 +1,239 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+        }
+
+    }
+}