You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by al...@apache.org on 2019/08/19 16:49:58 UTC

[nifi-minifi] branch master updated: Provenance reporting config can now be overridden using values set in the bootstrap.conf

This is an automated email from the ASF dual-hosted git repository.

aldrin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi-minifi.git


The following commit(s) were added to refs/heads/master by this push:
     new 2d6cdca  Provenance reporting config can now be overridden using values set in the bootstrap.conf
2d6cdca is described below

commit 2d6cdca142b15f83f144116f1ac5744dafc488a4
Author: GCHQ-NiFi <gc...@gmail.com>
AuthorDate: Fri Aug 16 09:44:09 2019 +0100

    Provenance reporting config can now be overridden using values set in the bootstrap.conf
    
    This closes #166.
    
    Signed-off-by: Aldrin Piri <al...@apache.org>
---
 .../apache/nifi/minifi/bootstrap/RunMiNiFi.java    | 62 +++++++++++++-
 .../minifi/bootstrap/util/ConfigTransformer.java   | 19 ++++-
 .../nifi/minifi/bootstrap/RunMiNiFiTest.java       | 27 ++++++
 .../bootstrap.conf.configured                      | 44 ++++++----
 .../bootstrap.conf.default                         | 18 +++-
 .../bootstrap-provenance-reporting/config.yml      | 97 ++++++++++++++++++++++
 .../nifi/minifi/commons/schema/ConfigSchema.java   |  4 +
 .../src/main/markdown/System_Admin_Guide.md        | 31 ++++---
 .../src/main/resources/conf/bootstrap.conf         | 10 +++
 9 files changed, 273 insertions(+), 39 deletions(-)

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 26fbf53..cc59f68 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
@@ -23,6 +23,7 @@ import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeExcepti
 import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
 import org.apache.nifi.minifi.bootstrap.status.PeriodicStatusReporter;
 import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
+import org.apache.nifi.minifi.commons.schema.ProvenanceReportingSchema;
 import org.apache.nifi.minifi.commons.schema.SecurityPropertiesSchema;
 import org.apache.nifi.minifi.commons.schema.SensitivePropsSchema;
 import org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys;
@@ -83,6 +84,11 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
 import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.commons.schema.RemoteProcessGroupSchema.TIMEOUT_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.COMMENT_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.SCHEDULING_PERIOD_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.SCHEDULING_STRATEGY_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.USE_COMPRESSION_KEY;
 
 /**
  * <p>
@@ -167,6 +173,27 @@ public class RunMiNiFi implements QueryableStatusAggregator, ConfigurationFileHo
                     SENSITIVE_PROPS_ALGORITHM_KEY,
                     SENSITIVE_PROPS_PROVIDER_KEY));
 
+    public static final String PROVENANCE_REPORTING_COMMENT_KEY = "nifi.minifi.provenance.reporting.comment";
+    public static final String PROVENANCE_REPORTING_SCHEDULING_STRATEGY_KEY = "nifi.minifi.provenance.reporting.scheduling.strategy";
+    public static final String PROVENANCE_REPORTING_SCHEDULING_PERIOD_KEY = "nifi.minifi.provenance.reporting.scheduling.period";
+    public static final String PROVENANCE_REPORTING_DESTINATION_URL_KEY = "nifi.minifi.provenance.reporting.destination.url";
+    public static final String PROVENANCE_REPORTING_INPUT_PORT_NAME_KEY = "nifi.minifi.provenance.reporting.input.port.name";
+    public static final String PROVENANCE_REPORTING_INSTANCE_URL_KEY = "nifi.minifi.provenance.reporting.instance.url";
+    public static final String PROVENANCE_REPORTING_COMPRESS_EVENTS_KEY = "nifi.minifi.provenance.reporting.compress.events";
+    public static final String PROVENANCE_REPORTING_BATCH_SIZE_KEY = "nifi.minifi.provenance.reporting.batch.size";
+    public static final String PROVENANCE_REPORTING_COMMUNICATIONS_TIMEOUT_KEY = "nifi.minifi.provenance.reporting.communications.timeout";
+
+    public static final Set<String> BOOTSTRAP_PROVENANCE_REPORTING_KEYS = new HashSet<>(
+            Arrays.asList(PROVENANCE_REPORTING_COMMENT_KEY,
+                    PROVENANCE_REPORTING_SCHEDULING_STRATEGY_KEY,
+                    PROVENANCE_REPORTING_SCHEDULING_PERIOD_KEY,
+                    PROVENANCE_REPORTING_DESTINATION_URL_KEY,
+                    PROVENANCE_REPORTING_INPUT_PORT_NAME_KEY,
+                    PROVENANCE_REPORTING_INSTANCE_URL_KEY,
+                    PROVENANCE_REPORTING_COMPRESS_EVENTS_KEY,
+                    PROVENANCE_REPORTING_BATCH_SIZE_KEY,
+                    PROVENANCE_REPORTING_COMMUNICATIONS_TIMEOUT_KEY
+            ));
 
     public static final Map<String, String> BOOTSTRAP_KEYS_TO_YML_KEYS;
 
@@ -187,6 +214,16 @@ public class RunMiNiFi implements QueryableStatusAggregator, ConfigurationFileHo
         mutableMap.put(SENSITIVE_PROPS_ALGORITHM_KEY, SensitivePropsSchema.SENSITIVE_PROPS_ALGORITHM_KEY);
         mutableMap.put(SENSITIVE_PROPS_PROVIDER_KEY, SensitivePropsSchema.SENSITIVE_PROPS_PROVIDER_KEY);
 
+        mutableMap.put(PROVENANCE_REPORTING_COMMENT_KEY, COMMENT_KEY);
+        mutableMap.put(PROVENANCE_REPORTING_SCHEDULING_STRATEGY_KEY, SCHEDULING_STRATEGY_KEY);
+        mutableMap.put(PROVENANCE_REPORTING_SCHEDULING_PERIOD_KEY, SCHEDULING_PERIOD_KEY);
+        mutableMap.put(PROVENANCE_REPORTING_DESTINATION_URL_KEY, ProvenanceReportingSchema.DESTINATION_URL_KEY);
+        mutableMap.put(PROVENANCE_REPORTING_INPUT_PORT_NAME_KEY, ProvenanceReportingSchema.PORT_NAME_KEY);
+        mutableMap.put(PROVENANCE_REPORTING_INSTANCE_URL_KEY, ProvenanceReportingSchema.ORIGINATING_URL_KEY);
+        mutableMap.put(PROVENANCE_REPORTING_COMPRESS_EVENTS_KEY, USE_COMPRESSION_KEY);
+        mutableMap.put(PROVENANCE_REPORTING_BATCH_SIZE_KEY, ProvenanceReportingSchema.BATCH_SIZE_KEY);
+        mutableMap.put(PROVENANCE_REPORTING_COMMUNICATIONS_TIMEOUT_KEY, TIMEOUT_KEY);
+
         BOOTSTRAP_KEYS_TO_YML_KEYS = Collections.unmodifiableMap(mutableMap);
     }
 
@@ -1765,7 +1802,8 @@ public class RunMiNiFi implements QueryableStatusAggregator, ConfigurationFileHo
             ConfigTransformer.transformConfigFile(
                 teeInputStream,
                 configDestinationPath,
-                buildSecurityPropertiesFromBootstrap(getBootstrapProperties()).orElse(null)
+                buildSecurityPropertiesFromBootstrap(getBootstrapProperties()).orElse(null),
+                buildProvenanceReportingPropertiesFromBootstrap(getBootstrapProperties()).orElse(null)
             );
 
             return ByteBuffer.wrap(byteArrayOutputStream.toByteArray());
@@ -1776,7 +1814,7 @@ public class RunMiNiFi implements QueryableStatusAggregator, ConfigurationFileHo
         }
     }
 
-    // TODO extract this to separate class BootstrapTransformer, and make private
+    // TODO extract this and buildProvenanceReportingPropertiesFromBootstrap to separate class BootstrapTransformer, and make private
     public Optional<SecurityPropertiesSchema> buildSecurityPropertiesFromBootstrap(final Properties bootstrapProperties) {
 
         Optional<SecurityPropertiesSchema> securityPropsOptional = Optional.empty();
@@ -1808,6 +1846,26 @@ public class RunMiNiFi implements QueryableStatusAggregator, ConfigurationFileHo
         return securityPropsOptional;
     }
 
+    public Optional<ProvenanceReportingSchema> buildProvenanceReportingPropertiesFromBootstrap(final Properties bootstrapProperties) {
+
+        Optional<ProvenanceReportingSchema> provenanceReportingPropsOptional = Optional.empty();
+
+        final Map<String, Object> provenanceReportingProperties = new HashMap<>();
+
+        BOOTSTRAP_PROVENANCE_REPORTING_KEYS.stream()
+                .filter(key -> StringUtils.isNotBlank(bootstrapProperties.getProperty(key)))
+                .forEach(key ->
+                        provenanceReportingProperties.put(BOOTSTRAP_KEYS_TO_YML_KEYS.get(key), bootstrapProperties.getProperty(key))
+                );
+
+        if (!provenanceReportingProperties.isEmpty()) {
+            final ProvenanceReportingSchema provenanceReportingSchema = new ProvenanceReportingSchema(provenanceReportingProperties);
+            provenanceReportingPropsOptional = Optional.of(provenanceReportingSchema);
+        }
+
+        return provenanceReportingPropsOptional;
+    }
+
     private static class Status {
 
         private final Integer port;
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
index 06856d0..7ba6fe3 100644
--- 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
@@ -95,16 +95,26 @@ public final class ConfigTransformer {
         final File ymlConfigFile = new File(sourceFile);
         final InputStream ios = new FileInputStream(ymlConfigFile);
 
-        transformConfigFile(ios, destPath, securityProperties);
+        transformConfigFile(ios, destPath, securityProperties, null);
     }
 
+    public static void transformConfigFile(String sourceFile, String destPath, SecurityPropertiesSchema securityProperties, ProvenanceReportingSchema provenanceReportingProperties) throws Exception {
+        final File ymlConfigFile = new File(sourceFile);
+        final InputStream ios = new FileInputStream(ymlConfigFile);
+
+        transformConfigFile(ios, destPath, securityProperties, provenanceReportingProperties);
+    }
 
     public static void transformConfigFile(InputStream sourceStream, String destPath) throws Exception {
-        transformConfigFile(sourceStream, destPath, null);
+        transformConfigFile(sourceStream, destPath, null, null);
     }
 
 
-    public static void transformConfigFile(InputStream sourceStream, String destPath, SecurityPropertiesSchema securityProperties) throws Exception {
+    public static void transformConfigFile(
+            InputStream sourceStream,
+            String destPath,
+            SecurityPropertiesSchema securityProperties,
+            ProvenanceReportingSchema provenanceReportingProperties) throws Exception {
         ConvertableSchema<ConfigSchema> convertableSchema = throwIfInvalid(SchemaLoader.loadConvertableSchemaFromYaml(sourceStream));
         ConfigSchema configSchema = throwIfInvalid(convertableSchema.convert());
 
@@ -112,6 +122,9 @@ public final class ConfigTransformer {
         if (securityProperties != null) {
             configSchema.setSecurityProperties(securityProperties);
         }
+        if (provenanceReportingProperties != null) {
+            configSchema.setProvenanceReportingProperties(provenanceReportingProperties);
+        }
 
         // Create nifi.properties and flow.xml.gz in memory
         ByteArrayOutputStream nifiPropertiesOutputStream = new ByteArrayOutputStream();
diff --git a/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/RunMiNiFiTest.java b/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/RunMiNiFiTest.java
index f6e0cb6..8bd651f 100644
--- a/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/RunMiNiFiTest.java
+++ b/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/RunMiNiFiTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.minifi.bootstrap;
 
+import org.apache.nifi.minifi.commons.schema.ProvenanceReportingSchema;
 import org.apache.nifi.minifi.commons.schema.SecurityPropertiesSchema;
 import org.apache.nifi.minifi.commons.schema.SensitivePropsSchema;
 import org.junit.Assert;
@@ -94,6 +95,32 @@ public class RunMiNiFiTest {
 
     }
 
+    @Test
+    public void buildProvenanceReportingNotDefined() throws Exception {
+        final RunMiNiFi testMiNiFi = new RunMiNiFi(null);
+        final Properties bootstrapProperties = getTestBootstrapProperties("bootstrap-provenance-reporting/bootstrap.conf.default");
+        final Optional<ProvenanceReportingSchema> provenanceReportingPropsOptional = testMiNiFi.buildProvenanceReportingPropertiesFromBootstrap(bootstrapProperties);
+        Assert.assertTrue(!provenanceReportingPropsOptional.isPresent());
+    }
+
+    @Test
+    public void buildProvenanceReportingDefined() throws Exception {
+        final RunMiNiFi testMiNiFi = new RunMiNiFi(null);
+        final Properties bootstrapProperties = getTestBootstrapProperties("bootstrap-provenance-reporting/bootstrap.conf.configured");
+        final Optional<ProvenanceReportingSchema> provenanceReportingPropsOptional = testMiNiFi.buildProvenanceReportingPropertiesFromBootstrap(bootstrapProperties);
+        Assert.assertTrue(provenanceReportingPropsOptional.isPresent());
+
+        final ProvenanceReportingSchema provenanceReportingSchema = provenanceReportingPropsOptional.get();
+        Assert.assertEquals("This is a comment!", provenanceReportingSchema.getComment());
+        Assert.assertEquals("TIMER_DRIVEN", provenanceReportingSchema.getSchedulingStrategy());
+        Assert.assertEquals("15 secs", provenanceReportingSchema.getSchedulingPeriod());
+        Assert.assertEquals("http://localhost:8080/", provenanceReportingSchema.getDestinationUrl());
+        Assert.assertEquals("provenance", provenanceReportingSchema.getPortName());
+        Assert.assertEquals("http://${hostname(true)}:8081/nifi", provenanceReportingSchema.getOriginatingUrl());
+        Assert.assertEquals("10 secs", provenanceReportingSchema.getTimeout());
+    }
+
+
     public static Properties getTestBootstrapProperties(final String fileName) throws IOException {
         final Properties bootstrapProperties = new Properties();
         try (final InputStream fis = RunMiNiFiTest.class.getClassLoader().getResourceAsStream(fileName)) {
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf b/minifi-bootstrap/src/test/resources/bootstrap-provenance-reporting/bootstrap.conf.configured
similarity index 78%
copy from minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf
copy to minifi-bootstrap/src/test/resources/bootstrap-provenance-reporting/bootstrap.conf.configured
index e3885aa..023a94e 100644
--- a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf
+++ b/minifi-bootstrap/src/test/resources/bootstrap-provenance-reporting/bootstrap.conf.configured
@@ -19,7 +19,7 @@
 java=java
 
 # Username to use when running MiNiFi. This value will be ignored on Windows.
-run.as=${minifi.run.as}
+run.as=
 
 # Configure where MiNiFi's lib and conf directories live
 # When running as a Windows service set full paths instead of relative paths
@@ -35,19 +35,29 @@ nifi.minifi.config=./conf/config.yml
 
 # Security Properties #
 # These properties take precedence over any equivalent properties specified in config.yml #
-nifi.minifi.security.keystore=
-nifi.minifi.security.keystoreType=
-nifi.minifi.security.keystorePasswd=
-nifi.minifi.security.keyPasswd=
-nifi.minifi.security.truststore=
-nifi.minifi.security.truststoreType=
-nifi.minifi.security.truststorePasswd=
-nifi.minifi.security.ssl.protocol=
-
-nifi.minifi.sensitive.props.key=
-nifi.minifi.sensitive.props.algorithm=
-nifi.minifi.sensitive.props.provider=
-
+nifi.minifi.security.keystore=/my/test/keystore.jks
+nifi.minifi.security.keystoreType=JKS
+nifi.minifi.security.keystorePasswd=mykeystorepassword
+nifi.minifi.security.keyPasswd=mykeypassword
+nifi.minifi.security.truststore=/my/test/truststore.jks
+nifi.minifi.security.truststoreType=JKS
+nifi.minifi.security.truststorePasswd=mytruststorepassword
+nifi.minifi.security.ssl.protocol=TLS
+
+nifi.minifi.sensitive.props.key=sensitivepropskey
+nifi.minifi.sensitive.props.algorithm=algo
+nifi.minifi.sensitive.props.provider=BC
+
+# Provenance Reporting Properties #
+# These properties take precedence over any equivalent properties specified in the config.yml #
+nifi.minifi.provenance.reporting.comment=This is a comment!
+nifi.minifi.provenance.reporting.scheduling.strategy=TIMER_DRIVEN
+nifi.minifi.provenance.reporting.scheduling.period=15 secs
+nifi.minifi.provenance.reporting.destination.url=http://localhost:8080/
+nifi.minifi.provenance.reporting.input.port.name=provenance
+nifi.minifi.provenance.reporting.instance.url=http://${hostname(true)}:8081/nifi
+nifi.minifi.provenance.reporting.batch.size=1000
+nifi.minifi.provenance.reporting.communications.timeout=10 secs
 
 # Notifiers to use for the associated agent, comma separated list of class names
 #nifi.minifi.notifier.ingestors=org.apache.nifi.minifi.bootstrap.configuration.ingestors.FileChangeIngestor
@@ -95,8 +105,8 @@ nifi.minifi.sensitive.props.provider=
 java.arg.1=-Dorg.apache.jasper.compiler.disablejsr199=true
 
 # JVM memory settings
-java.arg.2=-Xms${minifi.jvm.heap.mb}m
-java.arg.3=-Xmx${minifi.jvm.heap.mb}m
+java.arg.2=-Xms256m
+java.arg.3=-Xmx256m
 
 # Enable Remote Debugging
 #java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8000
@@ -116,4 +126,4 @@ java.arg.7=-Djava.security.egd=file:/dev/urandom
 #java.arg.13=-XX:+UseG1GC
 
 #Set headless mode by default
-java.arg.14=-Djava.awt.headless=true
+java.arg.14=-Djava.awt.headless=true
\ No newline at end of file
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf b/minifi-bootstrap/src/test/resources/bootstrap-provenance-reporting/bootstrap.conf.default
similarity index 88%
copy from minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf
copy to minifi-bootstrap/src/test/resources/bootstrap-provenance-reporting/bootstrap.conf.default
index e3885aa..ce8b6c4 100644
--- a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf
+++ b/minifi-bootstrap/src/test/resources/bootstrap-provenance-reporting/bootstrap.conf.default
@@ -19,7 +19,7 @@
 java=java
 
 # Username to use when running MiNiFi. This value will be ignored on Windows.
-run.as=${minifi.run.as}
+run.as=
 
 # Configure where MiNiFi's lib and conf directories live
 # When running as a Windows service set full paths instead of relative paths
@@ -48,6 +48,16 @@ nifi.minifi.sensitive.props.key=
 nifi.minifi.sensitive.props.algorithm=
 nifi.minifi.sensitive.props.provider=
 
+# Provenance Reporting Properties #
+# These properties take precedence over any equivalent properties specified in the config.yml #
+nifi.minifi.provenance.reporting.comment=
+nifi.minifi.provenance.reporting.scheduling.strategy=
+nifi.minifi.provenance.reporting.scheduling.period=
+nifi.minifi.provenance.reporting.destination.url=
+nifi.minifi.provenance.reporting.input.port.name=
+nifi.minifi.provenance.reporting.instance.url=
+nifi.minifi.provenance.reporting.batch.size=
+nifi.minifi.provenance.reporting.communications.timeout=
 
 # Notifiers to use for the associated agent, comma separated list of class names
 #nifi.minifi.notifier.ingestors=org.apache.nifi.minifi.bootstrap.configuration.ingestors.FileChangeIngestor
@@ -95,8 +105,8 @@ nifi.minifi.sensitive.props.provider=
 java.arg.1=-Dorg.apache.jasper.compiler.disablejsr199=true
 
 # JVM memory settings
-java.arg.2=-Xms${minifi.jvm.heap.mb}m
-java.arg.3=-Xmx${minifi.jvm.heap.mb}m
+java.arg.2=-Xms256m
+java.arg.3=-Xmx256m
 
 # Enable Remote Debugging
 #java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8000
@@ -116,4 +126,4 @@ java.arg.7=-Djava.security.egd=file:/dev/urandom
 #java.arg.13=-XX:+UseG1GC
 
 #Set headless mode by default
-java.arg.14=-Djava.awt.headless=true
+java.arg.14=-Djava.awt.headless=true
\ No newline at end of file
diff --git a/minifi-bootstrap/src/test/resources/bootstrap-provenance-reporting/config.yml b/minifi-bootstrap/src/test/resources/bootstrap-provenance-reporting/config.yml
new file mode 100644
index 0000000..ca062f5
--- /dev/null
+++ b/minifi-bootstrap/src/test/resources/bootstrap-provenance-reporting/config.yml
@@ -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.
+
+MiNiFi Config Version: 3
+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
+  max concurrent threads: 1
+  variable registry properties: additional.properties
+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
+Provenance Repository:
+  provenance rollover time: 1 min
+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
+Processors:
+- id: 94b8e610-b4ed-3ec9-b26f-c839931bf3e2
+  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
+    Initial Start Position: Beginning of File
+    Rolling Filename Pattern: minifi-app*
+Connections:
+- id: f6cef9b0-8982-391c-8d70-76f33917ac12
+  name: TailToS2S
+  source id: 94b8e610-b4ed-3ec9-b26f-c839931bf3e2
+  source relationship names:
+  - success
+  destination id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+  max work queue size: 0
+  max work queue data size: 1 MB
+  flowfile expiration: 60 sec
+  queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+Remote Process Groups:
+- id: 94b8e610-b4ed-3ec9-b26f-c839931bf3e3
+  name: NiFi Flow
+  url: https://localhost:8090/nifi
+  comment: ''
+  timeout: 30 secs
+  yield period: 10 sec
+  Input Ports:
+  - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+    name: tailed log
+    comment: ''
+    max concurrent tasks: 1
+    use compression: false
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ConfigSchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ConfigSchema.java
index e043697..ff9c9bf 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ConfigSchema.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ConfigSchema.java
@@ -195,6 +195,10 @@ public class ConfigSchema extends BaseSchema implements WritableSchema, Converta
         this.securityProperties = securityProperties;
     }
 
+    public void setProvenanceReportingProperties(ProvenanceReportingSchema provenanceReportingProperties) {
+        this.provenanceReportingProperties = provenanceReportingProperties;
+    }
+
     public ProcessGroupSchema getProcessGroupSchema() {
         return processGroupSchema;
     }
diff --git a/minifi-docs/src/main/markdown/System_Admin_Guide.md b/minifi-docs/src/main/markdown/System_Admin_Guide.md
index f2d694f..9546e97 100644
--- a/minifi-docs/src/main/markdown/System_Admin_Guide.md
+++ b/minifi-docs/src/main/markdown/System_Admin_Guide.md
@@ -618,22 +618,27 @@ When connecting via Site to Site, MiNiFi needs to know which input or output por
 
 ## 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  [...]
-`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.
+MiNiFi is currently designed only to report provenance data using the Site to Site protocol. 
 
 **Note:** In order to send via HTTPS, the "Security Properties" must be fully configured. A StandardSSLContextService will be made automatically with the ID "SSL-Context-Service" and used by the Provenance Reporting.
 
+Provenance Reporting can be configured in two ways via:
+
+* **config.yml**:  These properties are specified below and an example can be found in the Example Config File section.
+* **bootstrap.conf**:   This is an alternative means of configuration that takes precedence over the config.yml configuration and is a way of separating provenance reporting settings from the processing flow.
+
+*bootstrap.conf Property*                                 | *config.yml Property* | *Description*
+--------------------------------------------------------- | --------------------- | -------------------------------------------------------------------------------------------
+`nifi.minifi.provenance.reporting.comment`                | `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.
+`nifi.minifi.provenance.reporting.scheduling.strategy`    | `scheduling strategy` | The strategy for executing the Reporting Task. Valid options are CRON_DRIVEN or TIMER_DRIVEN
+`nifi.minifi.provenance.reporting.scheduling.period`      | `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  [...]
+`nifi.minifi.provenance.reporting.destination.url`        | `destination url`     | The URL to post the Provenance Events to.
+`nifi.minifi.provenance.reporting.input.port.name`        | `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.
+`nifi.minifi.provenance.reporting.instance.url`           | `originating url`     | The URL of this MiNiFi instance. This is used to include the Content URI to send to the destination.
+`nifi.minifi.provenance.reporting.compress.events`        | `use compression`     | Indicates whether or not to compress the events when being sent.
+`nifi.minifi.provenance.reporting.batch.size`             | `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.
+`nifi.minifi.provenance.reporting.communications.timeout` | `timeout`             | How long MiNiFi should wait before timing out the connection.
+
 ## NiFi Properties Overrides
 
 This is a yaml map that contains values to be put into *nifi.properties*.  This will supercede any hardcoded or other schema values that are substituted into *nifi.properties* file.
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf
index e3885aa..c8c40cf 100644
--- a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf
@@ -48,6 +48,16 @@ nifi.minifi.sensitive.props.key=
 nifi.minifi.sensitive.props.algorithm=
 nifi.minifi.sensitive.props.provider=
 
+# Provenance Reporting Properties #
+# These properties take precedence over any equivalent properties specified in the config.yml #
+nifi.minifi.provenance.reporting.comment=
+nifi.minifi.provenance.reporting.scheduling.strategy=
+nifi.minifi.provenance.reporting.scheduling.period=
+nifi.minifi.provenance.reporting.destination.url=
+nifi.minifi.provenance.reporting.input.port.name=
+nifi.minifi.provenance.reporting.instance.url=
+nifi.minifi.provenance.reporting.batch.size=
+nifi.minifi.provenance.reporting.communications.timeout=
 
 # Notifiers to use for the associated agent, comma separated list of class names
 #nifi.minifi.notifier.ingestors=org.apache.nifi.minifi.bootstrap.configuration.ingestors.FileChangeIngestor