You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jp...@apache.org on 2016/04/17 06:08:10 UTC

nifi-minifi git commit: MINIFI-15 Created a config file format with documentation, and a Util class to transform prospective config.yml into flow.xml and nifi.properties

Repository: nifi-minifi
Updated Branches:
  refs/heads/master 61e3a925d -> 3bbd3917c


MINIFI-15 Created a config file format with documentation, and a Util class to transform prospective config.yml into flow.xml and nifi.properties

This closes #8


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

Branch: refs/heads/master
Commit: 3bbd3917c1fcecc3a766fa3de721bcef83eac8ac
Parents: 61e3a92
Author: Joseph Percivall <jo...@yahoo.com>
Authored: Wed Mar 30 14:14:15 2016 -0400
Committer: Joseph Percivall <jo...@yahoo.com>
Committed: Sun Apr 17 00:06:43 2016 -0400

----------------------------------------------------------------------
 minifi-assembly/pom.xml                         |   4 +
 minifi-bootstrap/pom.xml                        |  11 +
 .../apache/nifi/minifi/bootstrap/RunMiNiFi.java |  11 +-
 .../bootstrap/util/ConfigTransformer.java       | 566 +++++++++++++++++++
 .../bootstrap/util/TestConfigTransformer.java   |  64 +++
 minifi-bootstrap/src/test/resources/config.yml  | 104 ++++
 minifi-docs/Properties_Guide.md                 | 315 +++++++++++
 .../src/main/resources/conf/config.yml          | 101 ++++
 .../reporting/ProvenanceReportingTask.java      |  15 +-
 9 files changed, 1179 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/3bbd3917/minifi-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-assembly/pom.xml b/minifi-assembly/pom.xml
index 8dd223a..7a0e6b5 100644
--- a/minifi-assembly/pom.xml
+++ b/minifi-assembly/pom.xml
@@ -165,6 +165,10 @@ limitations under the License.
             <artifactId>nifi-update-attribute-nar</artifactId>
             <type>nar</type>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+        </dependency>
     </dependencies>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/3bbd3917/minifi-bootstrap/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/pom.xml b/minifi-bootstrap/pom.xml
index 1c3e967..613efbe 100644
--- a/minifi-bootstrap/pom.xml
+++ b/minifi-bootstrap/pom.xml
@@ -45,5 +45,16 @@ limitations under the License.
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-expression-language</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.yaml</groupId>
+            <artifactId>snakeyaml</artifactId>
+            <version>1.17</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-core</artifactId>
+            <version>0.6.0</version>
+        </dependency>
     </dependencies>
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/3bbd3917/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/RunMiNiFi.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/RunMiNiFi.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/RunMiNiFi.java
index f37b7f3..0daa43d 100644
--- a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/RunMiNiFi.java
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/RunMiNiFi.java
@@ -908,13 +908,6 @@ public class RunMiNiFi {
         final Runtime runtime = Runtime.getRuntime();
         runtime.addShutdownHook(shutdownHook);
 
-        final String hostname = getHostname();
-        String now = sdf.format(System.currentTimeMillis());
-        String user = System.getProperty("user.name");
-        if (user == null || user.trim().isEmpty()) {
-            user = "Unknown User";
-        }
-
         return  new Tuple<ProcessBuilder,Process>(builder,process);
     }
 
@@ -922,6 +915,10 @@ public class RunMiNiFi {
     public void start() throws IOException, InterruptedException {
 
         Tuple<ProcessBuilder,Process> tuple = startMiNiFi();
+        if (tuple == null) {
+            cmdLogger.info("Start method returned null, ending start command.");
+            return;
+        }
         ProcessBuilder builder = tuple.getKey();
         Process process = tuple.getValue();
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/3bbd3917/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java
new file mode 100644
index 0000000..95e9ce0
--- /dev/null
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.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.minifi.bootstrap.util;
+
+
+import org.apache.nifi.controller.FlowSerializationException;
+import org.w3c.dom.DOMException;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.yaml.snakeyaml.Yaml;
+
+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 java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.io.UnsupportedEncodingException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.zip.GZIPOutputStream;
+
+public final class ConfigTransformer {
+    // Underlying version NIFI POC will be using
+    public static final String NIFI_VERSION = "0.6.0";
+
+    public static final String NAME_KEY = "name";
+    public static final String COMMENT_KEY = "comment";
+    public static final String ALWAYS_SYNC_KEY = "always sync";
+    public static final String YIELD_PERIOD_KEY = "yield period";
+    public static final String MAX_CONCURRENT_TASKS_KEY = "max concurrent tasks";
+    public static final String ID_KEY = "id";
+
+    public static final String FLOW_CONTROLLER_PROPS_KEY = "Flow Controller";
+
+    public static final String CORE_PROPS_KEY = "Core Properties";
+    public static final String FLOW_CONTROLLER_SHUTDOWN_PERIOD_KEY = "flow controller graceful shutdown period";
+    public static final String FLOW_SERVICE_WRITE_DELAY_INTERVAL_KEY = "flow service write delay interval";
+    public static final String ADMINISTRATIVE_YIELD_DURATION_KEY = "administrative yield duration";
+    public static final String BORED_YIELD_DURATION_KEY = "bored yield duration";
+
+    public static final String FLOWFILE_REPO_KEY = "FlowFile Repository";
+    public static final String PARTITIONS_KEY = "partitions";
+    public static final String CHECKPOINT_INTERVAL_KEY = "checkpoint interval";
+    public static final String THRESHOLD_KEY = "queue swap threshold";
+    public static final String SWAP_PROPS_KEY = "Swap";
+    public static final String IN_PERIOD_KEY = "in period";
+    public static final String IN_THREADS_KEY = "in threads";
+    public static final String OUT_PERIOD_KEY = "out period";
+    public static final String OUT_THREADS_KEY = "out threads";
+
+
+    public static final String CONTENT_REPO_KEY = "Content Repository";
+    public static final String CONTENT_CLAIM_MAX_APPENDABLE_SIZE_KEY = "content claim max appendable size";
+    public static final String CONTENT_CLAIM_MAX_FLOW_FILES_KEY = "content claim max flow files";
+
+    public static final String COMPONENT_STATUS_REPO_KEY = "Component Status Repository";
+    public static final String BUFFER_SIZE_KEY = "buffer size";
+    public static final String SNAPSHOT_FREQUENCY_KEY = "snapshot frequency";
+
+    public static final String SECURITY_PROPS_KEY = "Security Properties";
+    public static final String KEYSTORE_KEY = "keystore";
+    public static final String KEYSTORE_TYPE_KEY = "keystore type";
+    public static final String KEYSTORE_PASSWORD_KEY = "keystore password";
+    public static final String KEY_PASSWORD_KEY = "key password";
+    public static final String TRUSTSTORE_KEY = "truststore";
+    public static final String TRUSTSTORE_TYPE_KEY = "truststore type";
+    public static final String TRUSTSTORE_PASSWORD_KEY = "truststore password";
+    public static final String SENSITIVE_PROPS_KEY = "Sensitive Props";
+    public static final String SENSITIVE_PROPS_KEY__KEY = "key";
+    public static final String SENSITIVE_PROPS_ALGORITHM_KEY = "algorithm";
+    public static final String SENSITIVE_PROPS_PROVIDER_KEY = "provider";
+
+    public static final String PROCESSOR_CONFIG_KEY = "Processor Configuration";
+    public static final String CLASS_KEY = "class";
+    public static final String SCHEDULING_PERIOD_KEY = "scheduling period";
+    public static final String PENALIZATION_PERIOD_KEY = "penalization period";
+    public static final String SCHEDULING_STRATEGY_KEY = "scheduling strategy";
+    public static final String RUN_DURATION_NANOS_KEY = "run duration nanos";
+    public static final String AUTO_TERMINATED_RELATIONSHIPS_LIST_KEY = "auto-terminated relationships list";
+
+    public static final String PROCESSOR_PROPS_KEY = "Properties";
+
+    public static final String CONNECTION_PROPS_KEY = "Connection Properties";
+    public static final String MAX_WORK_QUEUE_SIZE_KEY = "max work queue size";
+    public static final String MAX_WORK_QUEUE_DATA_SIZE_KEY = "max work queue data size";
+    public static final String FLOWFILE_EXPIRATION__KEY = "flowfile expiration";
+    public static final String QUEUE_PRIORITIZER_CLASS_KEY = "queue prioritizer class";
+
+    public static final String REMOTE_PROCESSING_GROUP_KEY = "Remote Processing Group";
+    public static final String URL_KEY = "url";
+    public static final String TIMEOUT_KEY = "timeout";
+
+    public static final String INPUT_PORT_KEY = "Input Port";
+    public static final String USE_COMPRESSION_KEY = "use compression";
+
+    public static final String PROVENANCE_REPORTING_KEY = "Provenance Reporting";
+    public static final String DESTINATION_URL_KEY = "destination url";
+    public static final String PORT_NAME_KEY = "port name";
+    public static final String ORIGINATING_URL_KEY = "originating url";
+    public static final String BATCH_SIZE_KEY = "batch size";
+
+    public static final String SSL_PROTOCOL_KEY = "ssl protocol";
+
+    // Final util classes should have private constructor
+    private ConfigTransformer() {}
+
+    public static void transformConfigFile(String sourceFile, String destPath) throws Exception {
+        File ymlConfigFile = new File(sourceFile);
+        InputStream ios = new FileInputStream(ymlConfigFile);
+
+        transformConfigFile(ios, destPath);
+    }
+
+    public static void transformConfigFile(InputStream sourceStream, String destPath) throws Exception {
+        try {
+            Yaml yaml = new Yaml();
+
+            // Parse the YAML file
+            Map<String, Object> result = (Map<String, Object>) yaml.load(sourceStream);
+
+            // Write nifi.properties and flow.xml.gz
+            writeNiFiProperties(result, destPath);
+            writeFlowXml(result, destPath);
+        } finally {
+            if (sourceStream != null) {
+                sourceStream.close();
+            }
+        }
+    }
+
+    private static void writeNiFiProperties(Map<String, Object> topLevelYaml, String path) throws FileNotFoundException, UnsupportedEncodingException {
+        PrintWriter writer = null;
+        try {
+            writer = new PrintWriter(path+"nifi.properties", "UTF-8");
+
+            Map<String,Object> coreProperties = (Map<String, Object>) topLevelYaml.get(CORE_PROPS_KEY);
+            Map<String,Object> flowfileRepo = (Map<String, Object>) topLevelYaml.get(FLOWFILE_REPO_KEY);
+            Map<String, Object> swapProperties = (Map<String, Object>) flowfileRepo.get(SWAP_PROPS_KEY);
+            Map<String,Object> contentRepo = (Map<String, Object>) topLevelYaml.get(CONTENT_REPO_KEY);
+            Map<String,Object> componentStatusRepo = (Map<String, Object>) topLevelYaml.get(COMPONENT_STATUS_REPO_KEY);
+            Map<String,Object> securityProperties = (Map<String, Object>) topLevelYaml.get(SECURITY_PROPS_KEY);
+            Map<String,Object> sensitiveProperties = (Map<String, Object>) securityProperties.get(SENSITIVE_PROPS_KEY);
+
+
+            writer.print(PROPERTIES_FILE_APACHE_2_0_LICENSE);
+            writer.println("# Core Properties #");
+            writer.println();
+            writer.println("nifi.version="+NIFI_VERSION);
+            writer.println("nifi.flow.configuration.file=./conf/flow.xml.gz");
+            writer.println("nifi.flow.configuration.archive.dir=./conf/archive/");
+            writer.println("nifi.flowcontroller.autoResumeState=true");
+            writer.println("nifi.flowcontroller.graceful.shutdown.period=" + getValueString(coreProperties, FLOW_CONTROLLER_SHUTDOWN_PERIOD_KEY));
+            writer.println("nifi.flowservice.writedelay.interval=" + getValueString(coreProperties, FLOW_SERVICE_WRITE_DELAY_INTERVAL_KEY));
+            writer.println("nifi.administrative.yield.duration=" + getValueString(coreProperties, ADMINISTRATIVE_YIELD_DURATION_KEY));
+            writer.println("# If a component has no work to do (is \"bored\"), how long should we wait before checking again for work?");
+            writer.println("nifi.bored.yield.duration=" + getValueString(coreProperties, BORED_YIELD_DURATION_KEY));
+            writer.println();
+            writer.println("nifi.authority.provider.configuration.file=./conf/authority-providers.xml");
+            writer.println("nifi.login.identity.provider.configuration.file=./conf/login-identity-providers.xml");
+            writer.println("nifi.templates.directory=./conf/templates");
+            writer.println("nifi.ui.banner.text= ");
+            writer.println("nifi.ui.autorefresh.interval=30 sec");
+            writer.println("nifi.nar.library.directory=./lib");
+            writer.println("nifi.nar.working.directory=./work/nar/");
+            writer.println("nifi.documentation.working.directory=./work/docs/components");
+            writer.println();
+            writer.println("####################");
+            writer.println("# State Management #");
+            writer.println("####################");
+            writer.println("nifi.state.management.configuration.file=./conf/state-management.xml");
+            writer.println("# The ID of the local state provider");
+            writer.println("nifi.state.management.provider.local=local-provider");
+            writer.println();
+            writer.println("# H2 Settings");
+            writer.println("nifi.database.directory=./database_repository");
+            writer.println("nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE");
+            writer.println();
+            writer.println("# FlowFile Repository");
+            writer.println("nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository");
+            writer.println("nifi.flowfile.repository.directory=./flowfile_repository");
+            writer.println("nifi.flowfile.repository.partitions=" + getValueString(flowfileRepo, PARTITIONS_KEY));
+            writer.println("nifi.flowfile.repository.checkpoint.interval=" + getValueString(flowfileRepo,CHECKPOINT_INTERVAL_KEY));
+            writer.println("nifi.flowfile.repository.always.sync=" + getValueString(flowfileRepo,ALWAYS_SYNC_KEY));
+            writer.println();
+            writer.println("nifi.swap.manager.implementation=org.apache.nifi.controller.FileSystemSwapManager");
+            writer.println("nifi.queue.swap.threshold=" + getValueString(swapProperties, THRESHOLD_KEY));
+            writer.println("nifi.swap.in.period=" + getValueString(swapProperties, IN_PERIOD_KEY));
+            writer.println("nifi.swap.in.threads=" + getValueString(swapProperties, IN_THREADS_KEY));
+            writer.println("nifi.swap.out.period=" + getValueString(swapProperties, OUT_PERIOD_KEY));
+            writer.println("nifi.swap.out.threads=" + getValueString(swapProperties, OUT_THREADS_KEY));
+            writer.println();
+            writer.println("# Content Repository");
+            writer.println("nifi.content.repository.implementation=org.apache.nifi.controller.repository.FileSystemRepository");
+            writer.println("nifi.content.claim.max.appendable.size=" + getValueString(contentRepo, CONTENT_CLAIM_MAX_APPENDABLE_SIZE_KEY));
+            writer.println("nifi.content.claim.max.flow.files=" + getValueString(contentRepo, CONTENT_CLAIM_MAX_FLOW_FILES_KEY));
+            writer.println("nifi.content.repository.archive.max.retention.period=");
+            writer.println("nifi.content.repository.archive.max.usage.percentage=");
+            writer.println("nifi.content.repository.archive.enabled=false");
+            writer.println("nifi.content.repository.directory.default=./content_repository");
+            writer.println("nifi.content.repository.always.sync=" + getValueString(contentRepo, ALWAYS_SYNC_KEY));
+            writer.println();
+            writer.println("# Provenance Repository Properties");
+            writer.println("nifi.provenance.repository.implementation=org.apache.nifi.provenance.VolatileProvenanceRepository");
+            writer.println();
+            writer.println("# Volatile Provenance Respository Properties");
+            writer.println("nifi.provenance.repository.buffer.size=100000");
+            writer.println();
+            writer.println("# Component Status Repository");
+            writer.println("nifi.components.status.repository.implementation=org.apache.nifi.controller.status.history.VolatileComponentStatusRepository");
+            writer.println("nifi.components.status.repository.buffer.size=" + getValueString(componentStatusRepo, BUFFER_SIZE_KEY));
+            writer.println("nifi.components.status.snapshot.frequency=" + getValueString(componentStatusRepo, SNAPSHOT_FREQUENCY_KEY));
+            writer.println();
+            writer.println("# web properties #");
+            writer.println("nifi.web.war.directory=./lib");
+            writer.println("nifi.web.http.host=");
+            writer.println("nifi.web.http.port=8081");
+            writer.println("nifi.web.https.host=");
+            writer.println("nifi.web.https.port=");
+            writer.println("nifi.web.jetty.working.directory=./work/jetty");
+            writer.println("nifi.web.jetty.threads=200");
+            writer.println();
+            writer.println("# security properties #");
+            writer.println("nifi.sensitive.props.key=" + getValueString(sensitiveProperties, SENSITIVE_PROPS_KEY__KEY));
+            writer.println("nifi.sensitive.props.algorithm=" + getValueString(sensitiveProperties, SENSITIVE_PROPS_ALGORITHM_KEY));
+            writer.println("nifi.sensitive.props.provider=" + getValueString(sensitiveProperties, SENSITIVE_PROPS_PROVIDER_KEY));
+            writer.println();
+            writer.println("nifi.security.keystore=" + getValueString(securityProperties, KEYSTORE_KEY));
+            writer.println("nifi.security.keystoreType=" + getValueString(securityProperties, KEYSTORE_TYPE_KEY));
+            writer.println("nifi.security.keystorePasswd=" + getValueString(securityProperties, KEYSTORE_PASSWORD_KEY));
+            writer.println("nifi.security.keyPasswd=" + getValueString(securityProperties, KEY_PASSWORD_KEY));
+            writer.println("nifi.security.truststore=" + getValueString(securityProperties, TRUSTSTORE_KEY));
+            writer.println("nifi.security.truststoreType=" + getValueString(securityProperties, TRUSTSTORE_TYPE_KEY));
+            writer.println("nifi.security.truststorePasswd=" + getValueString(securityProperties, TRUSTSTORE_PASSWORD_KEY));
+            writer.println("nifi.security.needClientAuth=");
+            writer.println("nifi.security.user.credential.cache.duration=24 hours");
+            writer.println("nifi.security.user.authority.provider=file-provider");
+            writer.println("nifi.security.user.login.identity.provider=");
+            writer.println("nifi.security.support.new.account.requests=");
+            writer.println("# Valid Authorities include: ROLE_MONITOR,ROLE_DFM,ROLE_ADMIN,ROLE_PROVENANCE,ROLE_NIFI");
+            writer.println("nifi.security.anonymous.authorities=");
+            writer.println("nifi.security.ocsp.responder.url=");
+            writer.println("nifi.security.ocsp.responder.certificate=");
+            writer.println();
+            writer.println();
+            writer.println("# cluster node properties (only configure for cluster nodes) #");
+            writer.println("nifi.cluster.is.node=false");
+            writer.println();
+            writer.println("# cluster manager properties (only configure for cluster manager) #");
+            writer.println("nifi.cluster.is.manager=false");
+        } finally {
+            if (writer != null){
+                writer.flush();
+                writer.close();
+            }
+        }
+    }
+    private static void writeFlowXml(Map<String, Object> topLevelYaml, String path) throws Exception {
+        try {
+            // create a new, empty document
+            final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance();
+            docFactory.setNamespaceAware(true);
+
+            final DocumentBuilder docBuilder = docFactory.newDocumentBuilder();
+            final Document doc = docBuilder.newDocument();
+
+            // populate document with controller state
+            final Element rootNode = doc.createElement("flowController");
+            doc.appendChild(rootNode);
+            Map<String, Object> processorConfig = (Map<String, Object>) topLevelYaml.get(PROCESSOR_CONFIG_KEY);
+            addTextElement(rootNode, "maxTimerDrivenThreadCount", getValueString(processorConfig, MAX_CONCURRENT_TASKS_KEY));
+            addTextElement(rootNode, "maxEventDrivenThreadCount", getValueString(processorConfig, MAX_CONCURRENT_TASKS_KEY));
+            addProcessGroup(rootNode, topLevelYaml, "rootGroup");
+
+            Map<String, Object> securityProps = (Map<String, Object>) topLevelYaml.get(SECURITY_PROPS_KEY);
+            String sslAlgorithm = (String) securityProps.get(SSL_PROTOCOL_KEY);
+            if (sslAlgorithm != null && !(sslAlgorithm.isEmpty())) {
+                final Element controllerServicesNode = doc.createElement("controllerServices");
+                rootNode.appendChild(controllerServicesNode);
+                addSSLControllerService(controllerServicesNode, securityProps);
+            }
+
+            final Element reportingTasksNode = doc.createElement("reportingTasks");
+            rootNode.appendChild(reportingTasksNode);
+            addProvenanceReportingTask(reportingTasksNode, topLevelYaml);
+
+            final DOMSource domSource = new DOMSource(doc);
+            final OutputStream fileOut = Files.newOutputStream(Paths.get(path + "flow.xml.gz"));
+            final OutputStream outStream = new GZIPOutputStream(fileOut);
+            final StreamResult streamResult = new StreamResult(outStream);
+
+            // 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);
+            outStream.flush();
+            outStream.close();
+        } catch (final ParserConfigurationException | DOMException | TransformerFactoryConfigurationError | IllegalArgumentException | TransformerException e) {
+            throw new FlowSerializationException(e);
+        }
+    }
+
+    private static <K> String getValueString(Map<K,Object> map, K key){
+        Object value = map.get(key);
+        return value == null ? "" : value.toString();
+    }
+
+    private static void addSSLControllerService(final Element element, Map<String, Object> securityProperties) {
+        final Element serviceElement = element.getOwnerDocument().createElement("controllerService");
+        addTextElement(serviceElement, "id", "SSL-Context-Service");
+        addTextElement(serviceElement, "name", "SSL-Context-Service");
+        addTextElement(serviceElement, "comment", "");
+        addTextElement(serviceElement, "class", "org.apache.nifi.ssl.StandardSSLContextService");
+
+        addTextElement(serviceElement, "enabled", "true");
+
+        Map<String, Object> attributes = new HashMap<>();
+        attributes.put("Keystore Filename", securityProperties.get(KEYSTORE_KEY));
+        attributes.put("Keystore Type", securityProperties.get(KEYSTORE_TYPE_KEY));
+        attributes.put("Keystore Password", securityProperties.get(KEYSTORE_PASSWORD_KEY));
+        attributes.put("Truststore Filename", securityProperties.get(TRUSTSTORE_KEY));
+        attributes.put("Truststore Type", securityProperties.get(TRUSTSTORE_TYPE_KEY));
+        attributes.put("Truststore Password", securityProperties.get(TRUSTSTORE_PASSWORD_KEY));
+        attributes.put("SSL Protocol", securityProperties.get(SSL_PROTOCOL_KEY));
+
+        addConfiguration(serviceElement, attributes);
+
+        element.appendChild(serviceElement);
+    }
+
+    private static void addProcessGroup(final Element parentElement, Map<String, Object> topLevelYaml, final String elementName) {
+        Map<String,Object> flowControllerProperties = (Map<String, Object>) topLevelYaml.get(FLOW_CONTROLLER_PROPS_KEY);
+
+        final Document doc = parentElement.getOwnerDocument();
+        final Element element = doc.createElement(elementName);
+        parentElement.appendChild(element);
+        addTextElement(element, "id", "Root-Group");
+        addTextElement(element, "name", getValueString(flowControllerProperties, NAME_KEY) );
+        addPosition(element);
+        addTextElement(element, "comment", getValueString(flowControllerProperties, COMMENT_KEY));
+
+        Map<String,Object> processorConfig = (Map<String, Object>) topLevelYaml.get(PROCESSOR_CONFIG_KEY);
+        addProcessor(element, processorConfig);
+
+        Map<String,Object> remoteProcessingGroup = (Map<String, Object>) topLevelYaml.get(REMOTE_PROCESSING_GROUP_KEY);
+        addRemoteProcessGroup(element, remoteProcessingGroup);
+
+        addConnection(element, topLevelYaml);
+    }
+
+    private static void addProcessor(final Element parentElement, Map<String, Object> processorConfig) {
+
+        final Document doc = parentElement.getOwnerDocument();
+        final Element element = doc.createElement("processor");
+        parentElement.appendChild(element);
+        addTextElement(element, "id", "Processor");
+        addTextElement(element, "name", getValueString(processorConfig, NAME_KEY));
+
+        addPosition(element);
+        addStyle(element);
+
+        addTextElement(element, "comment", getValueString(processorConfig, COMMENT_KEY));
+        addTextElement(element, "class", getValueString(processorConfig, CLASS_KEY));
+        addTextElement(element, "maxConcurrentTasks", getValueString(processorConfig, MAX_CONCURRENT_TASKS_KEY));
+        addTextElement(element, "schedulingPeriod", getValueString(processorConfig, SCHEDULING_PERIOD_KEY));
+        addTextElement(element, "penalizationPeriod", getValueString(processorConfig, PENALIZATION_PERIOD_KEY));
+        addTextElement(element, "yieldPeriod", getValueString(processorConfig, YIELD_PERIOD_KEY));
+        addTextElement(element, "bulletinLevel", "WARN");
+        addTextElement(element, "lossTolerant", "false");
+        addTextElement(element, "scheduledState", "RUNNING");
+        addTextElement(element, "schedulingStrategy", getValueString(processorConfig, SCHEDULING_STRATEGY_KEY));
+        addTextElement(element, "runDurationNanos", getValueString(processorConfig, RUN_DURATION_NANOS_KEY));
+
+        addConfiguration(element, (Map<String, Object>) processorConfig.get(PROCESSOR_PROPS_KEY));
+
+        Collection<String> autoTerminatedRelationships = (Collection<String>) processorConfig.get(AUTO_TERMINATED_RELATIONSHIPS_LIST_KEY);
+        if (autoTerminatedRelationships != null) {
+            for (String rel : autoTerminatedRelationships) {
+                addTextElement(element, "autoTerminatedRelationship", rel);
+            }
+        }
+    }
+
+    private static void addProvenanceReportingTask(final Element element, Map<String, Object> topLevelYaml) {
+        Map<String, Object> provenanceProperties = (Map<String, Object>) topLevelYaml.get(PROVENANCE_REPORTING_KEY);
+        final Element taskElement = element.getOwnerDocument().createElement("reportingTask");
+        addTextElement(taskElement, "id", "Provenance-Reporting");
+        addTextElement(taskElement, "name", "Site-To-Site-Provenance-Reporting");
+        addTextElement(taskElement, "comment", getValueString(provenanceProperties, COMMENT_KEY));
+        addTextElement(taskElement, "class", "org.apache.nifi.minifi.provenance.reporting.ProvenanceReportingTask");
+        addTextElement(taskElement, "schedulingPeriod", getValueString(provenanceProperties, SCHEDULING_PERIOD_KEY));
+        addTextElement(taskElement, "scheduledState", "RUNNING");
+        addTextElement(taskElement, "schedulingStrategy", getValueString(provenanceProperties, SCHEDULING_STRATEGY_KEY));
+
+        Map<String, Object> attributes = new HashMap<>();
+        attributes.put("Destination URL", provenanceProperties.get(DESTINATION_URL_KEY));
+        attributes.put("Input Port Name", provenanceProperties.get(PORT_NAME_KEY));
+        attributes.put("MiNiFi URL", provenanceProperties.get(ORIGINATING_URL_KEY));
+        attributes.put("Compress Events", provenanceProperties.get(USE_COMPRESSION_KEY));
+        attributes.put("Batch Size", provenanceProperties.get(BATCH_SIZE_KEY));
+
+        Map<String, Object> securityProps = (Map<String, Object>) topLevelYaml.get(SECURITY_PROPS_KEY);
+        String sslAlgorithm = (String) securityProps.get(SSL_PROTOCOL_KEY);
+        if (sslAlgorithm != null && !(sslAlgorithm.isEmpty())) {
+            attributes.put("SSL Context Service", "SSL-Context-Service");
+        }
+
+        addConfiguration(taskElement, attributes);
+
+        element.appendChild(taskElement);
+    }
+
+    private static void addConfiguration(final Element element, Map<String, Object> elementConfig) {
+        final Document doc = element.getOwnerDocument();
+        for (final Map.Entry<String, Object> entry : elementConfig.entrySet()) {
+
+            final Element propElement = doc.createElement("property");
+            addTextElement(propElement, "name", entry.getKey());
+            if (entry.getValue() != null) {
+                addTextElement(propElement, "value", entry.getValue().toString());
+            }
+
+            element.appendChild(propElement);
+        }
+    }
+
+    private static void addStyle(final Element parentElement) {
+        final Element element = parentElement.getOwnerDocument().createElement("styles");
+        parentElement.appendChild(element);
+    }
+
+    private static void addRemoteProcessGroup(final Element parentElement, Map<String, Object> remoteProcessingGroup) {
+
+        final Document doc = parentElement.getOwnerDocument();
+        final Element element = doc.createElement("remoteProcessGroup");
+        parentElement.appendChild(element);
+        addTextElement(element, "id", "Remote-Process-Group");
+        addTextElement(element, "name", getValueString(remoteProcessingGroup, NAME_KEY));
+        addPosition(element);
+        addTextElement(element, "comment", getValueString(remoteProcessingGroup, COMMENT_KEY));
+        addTextElement(element, "url", getValueString(remoteProcessingGroup, URL_KEY));
+        addTextElement(element, "timeout", getValueString(remoteProcessingGroup, TIMEOUT_KEY));
+        addTextElement(element, "yieldPeriod", getValueString(remoteProcessingGroup, YIELD_PERIOD_KEY));
+        addTextElement(element, "transmitting", "true");
+
+        Map<String,Object> inputPort = (Map<String, Object>) remoteProcessingGroup.get(INPUT_PORT_KEY);
+        addRemoteGroupPort(element, inputPort, "inputPort");
+
+        parentElement.appendChild(element);
+    }
+
+    private static void addRemoteGroupPort(final Element parentElement, Map<String, Object> inputPort, final String elementName) {
+        final Document doc = parentElement.getOwnerDocument();
+        final Element element = doc.createElement(elementName);
+        parentElement.appendChild(element);
+        addTextElement(element, "id", getValueString(inputPort, ID_KEY));
+        addTextElement(element, "name", getValueString(inputPort, NAME_KEY));
+        addPosition(element);
+        addTextElement(element, "comments", getValueString(inputPort, COMMENT_KEY));
+        addTextElement(element, "scheduledState", "RUNNING");
+        addTextElement(element, "maxConcurrentTasks", getValueString(inputPort, MAX_CONCURRENT_TASKS_KEY));
+        addTextElement(element, "useCompression", getValueString(inputPort, USE_COMPRESSION_KEY));
+
+        parentElement.appendChild(element);
+    }
+
+    private static void addConnection(final Element parentElement, Map<String, Object> topLevelYaml) {
+        Map<String,Object> connectionProperties = (Map<String, Object>) topLevelYaml.get(CONNECTION_PROPS_KEY);
+        Map<String,Object> remoteProcessingGroup = (Map<String, Object>) topLevelYaml.get(REMOTE_PROCESSING_GROUP_KEY);
+        Map<String,Object> inputPort = (Map<String, Object>) remoteProcessingGroup.get(INPUT_PORT_KEY);
+        final Document doc = parentElement.getOwnerDocument();
+        final Element element = doc.createElement("connection");
+        parentElement.appendChild(element);
+        addTextElement(element, "id", "Connection");
+        addTextElement(element, "name", getValueString(connectionProperties, NAME_KEY));
+
+        final Element bendPointsElement = doc.createElement("bendPoints");
+        element.appendChild(bendPointsElement);
+
+        addTextElement(element, "labelIndex", "1");
+        addTextElement(element, "zIndex", "0");
+
+        addTextElement(element, "sourceId", "Processor");
+        addTextElement(element, "sourceGroupId", "Root-Group");
+        addTextElement(element, "sourceType", "PROCESSOR");
+
+        addTextElement(element, "destinationId", getValueString(inputPort,ID_KEY));
+        addTextElement(element, "destinationGroupId", "Remote-Process-Group");
+        addTextElement(element, "destinationType", "REMOTE_INPUT_PORT");
+
+        addTextElement(element, "relationship", "success");
+
+        addTextElement(element, "maxWorkQueueSize", getValueString(connectionProperties, MAX_WORK_QUEUE_SIZE_KEY));
+        addTextElement(element, "maxWorkQueueDataSize", getValueString(connectionProperties, MAX_WORK_QUEUE_DATA_SIZE_KEY));
+
+        addTextElement(element, "flowFileExpiration", getValueString(connectionProperties, FLOWFILE_EXPIRATION__KEY));
+        addTextElement(element, "queuePrioritizerClass", getValueString(connectionProperties, QUEUE_PRIORITIZER_CLASS_KEY));
+
+
+        parentElement.appendChild(element);
+    }
+
+    private static void addPosition(final Element parentElement) {
+        final Element element = parentElement.getOwnerDocument().createElement("position");
+        element.setAttribute("x", String.valueOf("0"));
+        element.setAttribute("y", String.valueOf("0"));
+        parentElement.appendChild(element);
+    }
+
+    private static 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);
+    }
+
+    public static final String PROPERTIES_FILE_APACHE_2_0_LICENSE =
+            "# Licensed to the Apache Software Foundation (ASF) under one or more\n" +
+            "# contributor license agreements.  See the NOTICE file distributed with\n" +
+            "# this work for additional information regarding copyright ownership.\n" +
+            "# The ASF licenses this file to You under the Apache License, Version 2.0\n" +
+            "# (the \"License\"); you may not use this file except in compliance with\n" +
+            "# the License.  You may obtain a copy of the License at\n" +
+            "#\n" +
+            "#     http://www.apache.org/licenses/LICENSE-2.0\n" +
+            "#\n" +
+            "# Unless required by applicable law or agreed to in writing, software\n" +
+            "# distributed under the License is distributed on an \"AS IS\" BASIS,\n" +
+            "# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n" +
+            "# See the License for the specific language governing permissions and\n" +
+            "# limitations under the License.\n"+
+            "\n";
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/3bbd3917/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/util/TestConfigTransformer.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/util/TestConfigTransformer.java b/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/util/TestConfigTransformer.java
new file mode 100644
index 0000000..cceed16
--- /dev/null
+++ b/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/util/TestConfigTransformer.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.minifi.bootstrap.util;
+
+import org.junit.Test;
+
+import java.io.File;
+import java.io.FileInputStream;
+
+import static org.junit.Assert.assertTrue;
+
+public class TestConfigTransformer {
+
+    @Test
+    public void doesTransformFile() throws Exception {
+
+        ConfigTransformer.transformConfigFile("./src/test/resources/config.yml", "./target/");
+        File nifiPropertiesFile = new File("./target/nifi.properties");
+
+        assertTrue(nifiPropertiesFile.exists());
+        assertTrue(nifiPropertiesFile.canRead());
+
+        nifiPropertiesFile.deleteOnExit();
+
+        File flowXml = new File("./target/flow.xml.gz");
+        assertTrue(flowXml.exists());
+        assertTrue(flowXml.canRead());
+
+        flowXml.deleteOnExit();
+    }
+
+    @Test
+    public void doesTransformInputStream() throws Exception {
+        File inputFile = new File("./src/test/resources/config.yml");
+        ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/");
+
+        File nifiPropertiesFile = new File("./target/nifi.properties");
+        assertTrue(nifiPropertiesFile.exists());
+        assertTrue(nifiPropertiesFile.canRead());
+
+        nifiPropertiesFile.deleteOnExit();
+
+        File flowXml = new File("./target/flow.xml.gz");
+        assertTrue(flowXml.exists());
+        assertTrue(flowXml.canRead());
+
+        flowXml.deleteOnExit();
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/3bbd3917/minifi-bootstrap/src/test/resources/config.yml
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/test/resources/config.yml b/minifi-bootstrap/src/test/resources/config.yml
new file mode 100644
index 0000000..0da4bf3
--- /dev/null
+++ b/minifi-bootstrap/src/test/resources/config.yml
@@ -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.
+
+Flow Controller:
+    name: MiNiFi Flow
+    comment:
+
+Core Properties:
+    flow controller graceful shutdown period: 10 sec
+    flow service write delay interval: 500 ms
+    administrative yield duration: 30 sec
+    bored yield duration: 10 millis
+
+FlowFile Repository:
+    partitions: 256
+    checkpoint interval: 2 mins
+    always sync: false
+    Swap:
+        threshold: 20000
+        in period: 5 sec
+        in threads: 1
+        out period: 5 sec
+        out threads: 4
+
+Content Repository:
+    content claim max appendable size: 10 MB
+    content claim max flow files: 100
+    always sync: false
+
+Component Status Repository:
+    buffer size: 1440
+    snapshot frequency: 1 min
+
+Security Properties:
+    keystore: /tmp/ssl/localhost-ks.jks
+    keystore type: JKS
+    keystore password: localtest
+    key password: localtest
+    truststore: /tmp/ssl/localhost-ts.jks
+    truststore type: JKS
+    truststore password: localtest
+    ssl protocol: TLS
+    Sensitive Props:
+        key:
+        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+        provider: BC
+
+Processor Configuration:
+    name: TailFile
+    class: org.apache.nifi.processors.standard.TailFile
+    max concurrent tasks: 1
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 1 sec
+    penalization period: 30 sec
+    yield period: 1 sec
+    run duration nanos: 0
+    auto-terminated relationships list:
+    Properties:
+        File to Tail: logs/minifi-app.log
+        Rolling Filename Pattern: minifi-app*
+        Initial Start Position: Beginning of File
+
+Connection Properties:
+    name:
+    max work queue size: 0
+    max work queue data size: 0 MB
+    flowfile expiration: 0 sec
+    queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+Remote Processing Group:
+    name: NiFi Flow
+    comment:
+    url: https://localhost:8090/nifi
+    timeout: 30 secs
+    yield period: 10 sec
+    Input Port:
+        id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+        name: tailed log
+        comments:
+        max concurrent tasks: 1
+        use compression: false
+
+Provenance Reporting:
+    comment:
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 30 sec
+    destination url: https://localhost:8090/
+    port name: provenance
+    originating url: http://${hostname(true)}:8081/nifi
+    use compression: true
+    timeout: 30 secs
+    batch size: 1000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/3bbd3917/minifi-docs/Properties_Guide.md
----------------------------------------------------------------------
diff --git a/minifi-docs/Properties_Guide.md b/minifi-docs/Properties_Guide.md
new file mode 100644
index 0000000..6306483
--- /dev/null
+++ b/minifi-docs/Properties_Guide.md
@@ -0,0 +1,315 @@
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+# MiNiFi System Administrator's Guide
+
+[Apache NiFi Team](dev@nifi.apache.org>)
+
+[NiFi Homepage](http://nifi.apache.org)
+
+
+# Config File
+
+The config.yml in the _conf_ directory is the main configuration file for controlling how MiNiFi runs. This section provides an overview of the properties in this file. The file is a YAML
+and follows the YAML format laid out [here](http://www.yaml.org/).
+
+NOTE: Note that values for periods of time and data sizes must include the unit of measure,
+for example "10 sec" or "10 MB", not simply "10".
+
+
+## Flow Controller
+
+The first section of config.yml is for naming and commenting on the file.
+
+ Property | Description
+ -------- | ---
+name      | The name of the file.
+comment   | A comment describing the usage of this config file.
+
+## Core Properties
+
+The Core Properties section applies to the core framework as a whole.
+
+*Property*                                 | *Description*
+---------------------------------------- | -----------
+flow controller graceful shutdown period | Indicates the shutdown period. The default value is 10 sec.
+flow service write delay interval        | When many changes are made to the flow.xml, this property specifies how long to wait before writing out the changes, so as to batch the changes into a single write. The default value is 500 ms.
+administrative yield duration            | If a component allows an unexpected exception to escape, it is considered a bug. As a result, the framework will pause (or administratively yield) the component for this amount of time. This is done so that the component does not use up massive amounts of system resources, since it is known to have problems in the existing state. The default value is 30 sec.
+bored yield duration                     | When a component has no work to do (i.e., is "bored"), this is the amount of time it will wait before checking to see if it has new data to work on. This way, it does not use up CPU resources by checking for new work too often. When setting this property, be aware that it could add extra latency for components that do not constantly have work to do, as once they go into this "bored" state, they will wait this amount of time before checking for more work. The default value is 10 millis.
+
+## FlowFile Repository
+
+The FlowFile repository keeps track of the attributes and current state of each FlowFile in the system. By default,
+this repository is installed in the same root installation directory as all the other repositories; however, it is advisable
+to configure it on a separate drive if available.
+
+*Property*  | *Description*
+----------  | ------------
+partitions  | The number of partitions. The default value is 256.
+checkpoint interval | The FlowFile Repository checkpoint interval. The default value is 2 mins.
+always sync | If set to _true_, any change to the repository will be synchronized to the disk, meaning that NiFi will ask the operating system not to cache the information. This is very expensive and can significantly reduce NiFi performance. However, if it is _false_, there could be the potential for data loss if either there is a sudden power loss or the operating system crashes. The default value is _false_.
+
+#### Swap Subsection
+
+A part of the FlowFile Repository section there is a Swap subsection.
+
+NiFi keeps FlowFile information in memory (the JVM)
+but during surges of incoming data, the FlowFile information can start to take up so much of the JVM that system performance
+suffers. To counteract this effect, NiFi "swaps" the FlowFile information to disk temporarily until more JVM space becomes
+available again. The "Swap" subsection of properties govern how that process occurs.
+
+*Property*  | *Description*
+----------  | ------------
+threshold   | The queue threshold at which NiFi starts to swap FlowFile information to disk. The default value is 20000.
+in period   | The swap in period. The default value is 5 sec.
+in threads  | The number of threads to use for swapping in. The default value is 1.
+out period  | The swap out period. The default value is 5 sec.
+out threads | The number of threads to use for swapping out. The default value is 4.
+
+## Content Repository
+
+The Content Repository holds the content for all the FlowFiles in the system. By default, it is installed in the same root
+installation directory as all the other repositories; however, administrators will likely want to configure it on a separate
+drive if available. If nothing else, it is best if the Content Repository is not on the same drive as the FlowFile Repository.
+In dataflows that handle a large amount of data, the Content Repository could fill up a disk and the
+FlowFile Repository, if also on that disk, could become corrupt. To avoid this situation, configure these repositories on different drives.
+
+*Property*                        | *Description*
+--------------------------------  | -------------
+content claim max appendable size | The maximum size for a content claim. The default value is 10 MB.
+content claim max flow files      | The maximum number of FlowFiles to assign to one content claim. The default value is 100.
+always sync                       | If set to _true_, any change to the repository will be synchronized to the disk, meaning that NiFi will ask the operating system not to cache the information. This is very expensive and can significantly reduce NiFi performance. However, if it is _false_, there could be the potential for data loss if either there is a sudden power loss or the operating system crashes. The default value is _false_.
+
+## *Component Status Repository*
+
+The Component Status Repository contains the information for the Component Status History tool in the User Interface. These
+properties govern how that tool works.
+
+The buffer.size and snapshot.frequency work together to determine the amount of historical data to retain. As an example to
+configure two days worth of historical data with a data point snapshot occurring every 5 minutes you would configure
+snapshot.frequency to be "5 mins" and the buffer.size to be "576". To further explain this example for every 60 minutes there
+are 12 (60 / 5) snapshot windows for that time period. To keep that data for 48 hours (12 * 48) you end up with a buffer size
+of 576.
+
+*Property*        | *Description*
+----------------- | -------------
+buffer size       | Specifies the buffer size for the Component Status Repository. The default value is 1440.
+snapshot frequency | This value indicates how often to present a snapshot of the components' status history. The default value is 1 min.
+
+## *Security Properties*
+
+These properties pertain to various security features in NiFi. Many of these properties are covered in more detail in the
+Security Configuration section of this Administrator's Guide.
+
+*Property*          | *Description*
+------------------- | -------------
+keystore            | The full path and name of the keystore. It is blank by default.
+keystore type       | The keystore type. It is blank by default.
+keystore password   | The keystore password. It is blank by default.
+key password        | The key password. It is blank by default.
+truststore          | The full path and name of the truststore. It is blank by default.
+truststore type     | The truststore type. It is blank by default.
+truststore password | The truststore password. It is blank by default.
+ssl protocol        | The protocol to use when communicating via https. Necessary to transfer provenance securely.
+
+#### Sensitive Properties Subsection
+
+Some properties for processors are marked as _sensitive_ and should be encrypted. These following properties will be used to encrypt the properties while in use by MiNiFi. This will currently *not* be used to encrypt properties in the config file.
+
+*Property* | *Description*
+---------- | -------------
+key        | This is the password used to encrypt any sensitive property values that are configured in processors. By default, it is blank, but the system administrator should provide a value for it. It can be a string of any length, although the recommended minimum length is 10 characters. Be aware that once this password is set and one or more sensitive processor properties have been configured, this password should not be changed.
+algorithm  | The algorithm used to encrypt sensitive properties. The default value is `PBEWITHMD5AND256BITAES-CBC-OPENSSL`.
+provider   | The sensitive property provider. The default value is BC.
+
+## Processor Configuration
+
+The current implementation of MiNiFi supports one source processor. These properties are the basic configuration general to all processor implementations.
+
+*Property*                          | *Description*
+----------------------------------- | -------------
+name                                | The name of what this processor will do. This is not used for any underlying implementation but solely for the users of this configuration and MiNiFi agent.
+class                               | The fully qualified java class name of the processor to run. For example for the standard TailFile processor it would be: org.apache.nifi.processors.standard.TailFile
+max concurrent tasks                | The maximum number of tasks that the processor will use.
+scheduling strategy                 | The strategy for executing the processor. Valid options are `CRON_DRIVEN` or `TIMER_DRIVEN`
+scheduling period                   | This property expects different input depending on the scheduling strategy selected. For the `TIMER_DRIVEN` scheduling strategy, this value is a time duration specified by a number followed by a time unit. For example, 1 second or 5 mins. The default value of 0 sec means that the Processor should run as often as possible as long as it has data to process. This is true for any time duration of 0, regardless of the time unit (i.e., 0 sec, 0 mins, 0 days). For an explanation of values that are applicable for the CRON driven scheduling strategy, see the description of the CRON driven scheduling strategy in the scheduling tab section of the [NiFi User documentation](https://nifi.apache.org/docs/nifi-docs/html/user-guide.html#scheduling-tab).
+penalization period                 | Specifies how long FlowFiles will be penalized.
+yield period                        | In the event the processor cannot make progress it should `yield` which will prevent the processor from being scheduled to run for some period of time. That period of time is specific using this property.
+run duration nanos                  | If the processor supports batching this property can be used to control how long the Processor should be scheduled to run each time that it is triggered. Smaller values will have lower latency but larger values will have higher throughput. This period should typically only be set between 0 and 2000000000 (2 seconds).
+auto-terminated relationships list  | A YAML list of the relationships to auto-terminate for the processor.
+
+#### Processor Properties
+
+Within the Processor Configuration section, there is the `Properties` subsection. The keys and values in this section are the property names and values for the processor. For example the TailFile processor would have a section like this:
+
+    Properties:
+        File to Tail: logs/nifi-app.log
+        Rolling Filename Pattern: nifi-app*
+        State File: ./conf/state/tail-file
+        Initial Start Position: Beginning of File
+
+## Connection Properties
+
+There is only one connection in this initial version of MiNiFi, between the source processor and the Remote Processing Group. These properties allow for customization of the of the connection.
+
+*Property*               | *Description*
+--------------------     | -------------
+name                     | The name of what this connection will do. This is not used for any underlying implementation but solely for the users of this configuration and MiNiFi agent.
+max work queue size      | This property is the max number of FlowFiles that can be in the queue before back pressure is applied. When back pressure is applied the source processor will no longer be scheduled to run.
+max work queue data size | This property specifies the maximum amount of data (in size) that should be queued up before applying back pressure.  When back pressure is applied the source processor will no longer be scheduled to run.
+flowfile expiration      | Indicates how long FlowFiles are allowed to exist in the connection before be expired (automatically removed from the flow).
+queue prioritizer class  | This configuration option specifies the fully qualified java class path of a queue prioritizer to use. If no special prioritizer is desired then it should be left blank. An example value of this property is: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+## Remote Processing Group
+
+MiNiFi is currently designed only to send back data to core NiFi instances using the Site to Site protocol. These properties configure the Remote Processing Group that links to that core instance.
+
+*Property*   | *Description*
+------------ | -------------
+name         | The name of what this Remote Processing Group points to. This is not used for any underlying implementation but solely for the users of this configuration and MiNiFi agent.
+comment      | A comment about the Remote Processing Group. This is not used for any underlying implementation but solely for the users of this configuration and MiNiFi agent.
+url          | The URL of the core NiFi instance.
+timeout      | How long MiNiFi should wait before timing out the connection.
+yield period | When communication with this Remote Processing Group fails, it will not be scheduled again for this amount of time.
+
+
+#### Input Port Subsection
+
+When connecting via Site to Site, MiNiFi needs to know which input port to communicate to of the core NiFi instance. These properties designate and configure communication with that port.
+
+*Property*           | *Description*
+-------------------- | -------------
+id                   | The id of the input port as it exists on the core NiFi instance. To get this information access the UI of the core instance, right the input port that is desired to be connect to and select "configure". The id of the port should under the "Id" section.
+name                 | The name of the input port as it exists on the core NiFi instance. To get this information access the UI of the core instance, right the input port that is desired to be connect to and select "configure". The id of the port should under the "Port name" section.
+comments:            | A comment about the Input Port. This is not used for any underlying implementation but solely for the users of this configuration and MiNiFi agent.
+max concurrent tasks | The number of tasks that this port should be scheduled for at maximum.
+use compression      | Whether or not compression should be used when communicating with the port. This is a boolean value of either "true" or "false"
+
+## Provenance Reporting
+
+MiNiFi is currently designed only to report provenance data using the Site to Site protocol. These properties configure the underlying reporting task that sends the provenance events.
+
+*Property*           | *Description*
+-------------------- | -------------
+comment              | A comment about the Provenance reporting. This is not used for any underlying implementation but solely for the users of this configuration and MiNiFi agent.
+scheduling strategy  | The strategy for executing the Reporting Task. Valid options are `CRON_DRIVEN` or `TIMER_DRIVEN`
+scheduling period    | This property expects different input depending on the scheduling strategy selected. For the `TIMER_DRIVEN` scheduling strategy, this value is a time duration specified by a number followed by a time unit. For example, 1 second or 5 mins. The default value of 0 sec means that the Processor should run as often as possible as long as it has data to process. This is true for any time duration of 0, regardless of the time unit (i.e., 0 sec, 0 mins, 0 days). For an explanation of values that are applicable for the CRON driven scheduling strategy, see the description of the CRON driven scheduling strategy in the scheduling tab section of the [NiFi User documentation](https://nifi.apache.org/docs/nifi-docs/html/user-guide.html#scheduling-tab).
+destination url      | The URL to post the Provenance Events to.
+port name            | The name of the input port as it exists on the receiving NiFi instance. To get this information access the UI of the core instance, right the input port that is desired to be connect to and select "configure". The id of the port should under the "Port name" section.
+originating url      | The URL of this MiNiFi instance. This is used to include the Content URI to send to the destination.
+use compression      | Indicates whether or not to compress the events when being sent.
+timeout              | How long MiNiFi should wait before timing out the connection.
+batch size           | Specifies how many records to send in a single batch, at most. This should be significantly above the expected amount of records generated between scheduling. If it is not, then there is the potential for the Provenance reporting to lag behind event generation and never catch up.
+
+
+
+# Example Config File
+
+Below is an example config YAML file to tail the minifi-app.log send it and provenance data back to a secure instance of NiFi.
+
+
+``` yaml
+Flow Controller:
+    name: MiNiFi Flow
+    comment:
+
+Core Properties:
+    flow controller graceful shutdown period: 10 sec
+    flow service write delay interval: 500 ms
+    administrative yield duration: 30 sec
+    bored yield duration: 10 millis
+
+FlowFile Repository:
+    partitions: 256
+    checkpoint interval: 2 mins
+    always sync: false
+    Swap:
+        threshold: 20000
+        in period: 5 sec
+        in threads: 1
+        out period: 5 sec
+        out threads: 4
+
+Content Repository:
+    content claim max appendable size: 10 MB
+    content claim max flow files: 100
+    always sync: false
+
+Component Status Repository:
+    buffer size: 1440
+    snapshot frequency: 1 min
+
+Security Properties:
+    keystore: /tmp/ssl/localhost-ks.jks
+    keystore type: JKS
+    keystore password: localtest
+    key password: localtest
+    truststore: /tmp/ssl/localhost-ts.jks
+    truststore type: JKS
+    truststore password: localtest
+    ssl protocol: TLS
+    Sensitive Props:
+        key:
+        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+        provider: BC
+
+Processor Configuration:
+    name: TailFile
+    class: org.apache.nifi.processors.standard.TailFile
+    max concurrent tasks: 1
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 1 sec
+    penalization period: 30 sec
+    yield period: 1 sec
+    run duration nanos: 0
+    auto-terminated relationships list:
+    Properties:
+        File to Tail: logs/minifi-app.log
+        Rolling Filename Pattern: minifi-app*
+        Initial Start Position: Beginning of File
+
+Connection Properties:
+    name:
+    max work queue size: 0
+    max work queue data size: 0 MB
+    flowfile expiration: 0 sec
+    queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+Remote Processing Group:
+    name: NiFi Flow
+    comment:
+    url: https://localhost:8090/nifi
+    timeout: 30 secs
+    yield period: 10 sec
+    Input Port:
+        id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+        name: tailed log
+        comments:
+        max concurrent tasks: 1
+        use compression: false
+
+Provenance Reporting:
+    comment:
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 30 sec
+    destination url: https://localhost:8090/
+    port name: provenance
+    originating url: http://${hostname(true)}:8081/nifi
+    use compression: true
+    timeout: 30 secs
+    batch size: 1000
+```

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/3bbd3917/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/config.yml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/config.yml b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/config.yml
new file mode 100644
index 0000000..064a746
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/config.yml
@@ -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.
+
+Flow Controller:
+    name: MiNiFi Flow
+    comment:
+
+Core Properties:
+    flow controller graceful shutdown period: 10 sec
+    flow service write delay interval: 500 ms
+    administrative yield duration: 30 sec
+    bored yield duration: 10 millis
+
+FlowFile Repository:
+    partitions: 256
+    checkpoint interval: 2 mins
+    always sync: false
+    Swap:
+        threshold: 20000
+        in period: 5 sec
+        in threads: 1
+        out period: 5 sec
+        out threads: 4
+
+Content Repository: 
+    content claim max appendable size: 10 MB
+    content claim max flow files: 100
+    always sync: false
+
+Component Status Repository:
+    buffer size: 1440
+    snapshot frequency: 1 min
+
+Security Properties:
+    keystore:
+    keystore type:
+    keystore password:
+    key password:
+    truststore:
+    truststore type:
+    truststore password:
+    ssl protocol:
+    Sensitive Props:
+        key:
+        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+        provider: BC
+
+Processor Configuration:
+    name:
+    class:
+    max concurrent tasks:
+    scheduling strategy:
+    scheduling period:
+    penalization period:
+    yield period:
+    run duration nanos:
+    auto-terminated relationships list:
+    Properties:
+
+Connection Properties:
+    name:
+    max work queue size: 0
+    max work queue data size: 0 MB
+    flowfile expiration: 0 sec
+    queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+Remote Processing Group:
+    name:
+    comment: 
+    url:
+    timeout:
+    yield period:
+    Input Port:
+        id:
+        name:
+        comments:
+        max concurrent tasks:
+        use compression:
+
+Provenance Reporting:
+    comment:
+    scheduling strategy:
+    scheduling period:
+    destination url:
+    port name:
+    originating url:
+    use compression:
+    timeout:
+    batch size:
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/3bbd3917/minifi-nar-bundles/minifi-provenance-reporting-bundle/minifi-provenance-reporting-task/src/main/java/org/apache/nifi/minifi/provenance/reporting/ProvenanceReportingTask.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-provenance-reporting-bundle/minifi-provenance-reporting-task/src/main/java/org/apache/nifi/minifi/provenance/reporting/ProvenanceReportingTask.java b/minifi-nar-bundles/minifi-provenance-reporting-bundle/minifi-provenance-reporting-task/src/main/java/org/apache/nifi/minifi/provenance/reporting/ProvenanceReportingTask.java
index 8ed5dee..a0d1776 100644
--- a/minifi-nar-bundles/minifi-provenance-reporting-bundle/minifi-provenance-reporting-task/src/main/java/org/apache/nifi/minifi/provenance/reporting/ProvenanceReportingTask.java
+++ b/minifi-nar-bundles/minifi-provenance-reporting-bundle/minifi-provenance-reporting-task/src/main/java/org/apache/nifi/minifi/provenance/reporting/ProvenanceReportingTask.java
@@ -243,12 +243,17 @@ public class ProvenanceReportingTask extends AbstractReportingTask {
 
         Long currMaxId = context.getEventAccess().getProvenanceRepository().getMaxEventId();
 
+        if(currMaxId == null) {
+            getLogger().debug("No events to send because no events have been created yet.");
+            return;
+        }
+
         if (firstEventId < 0) {
             Map<String, String> state;
             try {
                 state = context.getStateManager().getState(Scope.LOCAL).toMap();
             } catch (IOException e) {
-                getLogger().error("Failed to get state at start up due to {}", e);
+                getLogger().error("Failed to get state at start up due to {}:"+e.getMessage(), e);
                 return;
             }
             if (state.containsKey(LAST_EVENT_ID_KEY)) {
@@ -271,7 +276,7 @@ public class ProvenanceReportingTask extends AbstractReportingTask {
         try {
             events = context.getEventAccess().getProvenanceEvents(firstEventId, context.getProperty(BATCH_SIZE).asInteger());
         } catch (final IOException ioe) {
-            getLogger().error("Failed to retrieve Provenance Events from repository due to {}", ioe);
+            getLogger().error("Failed to retrieve Provenance Events from repository due to {}:"+ioe.getMessage(), ioe);
             return;
         }
 
@@ -322,7 +327,7 @@ public class ProvenanceReportingTask extends AbstractReportingTask {
             getLogger().info("Successfully sent {} Provenance Events to destination in {} ms; Transaction ID = {}; First Event ID = {}",
                 new Object[] {events.size(), transferMillis, transactionId, events.get(0).getEventId()});
         } catch (final IOException e) {
-            throw new ProcessException("Failed to send Provenance Events to destination due to IOException", e);
+            throw new ProcessException("Failed to send Provenance Events to destination due to IOException:" + e.getMessage(), e);
         }
 
         final ProvenanceEventRecord lastEvent = events.get(events.size() - 1);
@@ -334,8 +339,8 @@ public class ProvenanceReportingTask extends AbstractReportingTask {
             newMapOfState.put(LAST_EVENT_ID_KEY, lastEventId);
             stateManager.replace(stateMap, newMapOfState, Scope.LOCAL);
         } catch (final IOException ioe) {
-            getLogger().error("Failed to update state to {} due to {}; this could result in events being re-sent after a restart of MiNiFi",
-                new Object[] {lastEventId, ioe}, ioe);
+            getLogger().error("Failed to update state to {} due to {}; this could result in events being re-sent after a restart of MiNiFi. The message of {} was: {}",
+                new Object[] {lastEventId, ioe, ioe, ioe.getMessage()}, ioe);
         }
 
         firstEventId = lastEvent.getEventId() + 1;