You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2020/09/02 17:59:10 UTC

[nifi] branch support/nifi-1.12.x created (now 353a91b)

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

joewitt pushed a change to branch support/nifi-1.12.x
in repository https://gitbox.apache.org/repos/asf/nifi.git.


      at 353a91b  NIFI-7167 Fixing resource leaks in IdentifyMimeType

This branch includes the following new commits:

     new 6c2f59f  NIFI-7787 prepping versions for support/1.12.x branch starting at 1.12.1
     new 669054c  NIFI-7758: Avoid calling InetAddress.getHostName() because doing so results in a reverse DNS Lookup, which can be expensive
     new cf7bf8f  NIFI-7751 - This closes #4483. fix for TestExecuteStreamCommand unit tests
     new 6db63de  NIFI-7760 Remove invalid permission check
     new 996c169  NIFI-7766 Getting initialize class org.apache.nifi.util.text.RegexDateTimeMatcher error in jsontreereader
     new 4692381  NIFI-7714: QueryCassandra loses precision when converting timestamps to JSON
     new a173426  NIFI-7309 Update Admin Guide by removing unused properties and updating default values
     new 3ef212e  NIFI-6767 Persist registry URL after service restart.
     new f4bce4f  NIFI-7661 Change validation of controller service against it's API to use comparison of inteface methods instead of bundle coordinates
     new 87d65e7  NIFI-7762 - support copy-paste on Disabled ports
     new e061964  NIFI-7779: Avoid NPE when a ExecuteScript Processor has null value properties during a component search from canvas
     new c10bd49  NIFI-7740: Add Records Per Transaction and Transactions Per Batch properties to PutHive3Streaming
     new 5da33ee  NIFI-7778: Made corrections in descriptions of padLeft, padRight, plus (#4504)
     new e2bf1f2  NIFI-7767 - Fixed issue with tls-toolkit not adding SANs to generated certificates. Added tests. NIFI-7767 - Fixed up TlsCertificateAuthorityTest to include SAN in tests.
     new 29fc0a7  NIFI-7730 Added regression tests for multiple certificate keystores. Cleaned up JettyServer code. Changed test logging severity to include debug statements. Added test resources.
     new 353a91b  NIFI-7167 Fixing resource leaks in IdentifyMimeType

The 16 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[nifi] 03/16: NIFI-7751 - This closes #4483. fix for TestExecuteStreamCommand unit tests

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.12.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit cf7bf8f0b8b9b51858d0df0eec94522d2f4ab4e6
Author: Pierre Villard <pi...@gmail.com>
AuthorDate: Wed Aug 19 11:12:26 2020 +0200

    NIFI-7751 - This closes #4483. fix for TestExecuteStreamCommand unit tests
    
    Signed-off-by: Joe Witt <jo...@apache.org>
---
 .../org/apache/nifi/processors/standard/TestExecuteStreamCommand.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java
index f36ded4..dfa5b81 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java
@@ -274,7 +274,7 @@ public class TestExecuteStreamCommand {
         List<MockFlowFile> flowFiles = controller.getFlowFilesForRelationship(ExecuteStreamCommand.OUTPUT_STREAM_RELATIONSHIP);
         MockFlowFile flowFile = flowFiles.get(0);
         assertEquals(0, flowFile.getSize());
-        assertEquals("fffffffffffffffffffffffffffffff", flowFile.getAttribute("execution.error").substring(0, 31));
+        assertTrue(flowFile.getAttribute("execution.error").contains("fffffffffffffffffffffffffffffff"));
     }
 
     @Test
@@ -306,7 +306,7 @@ public class TestExecuteStreamCommand {
         List<MockFlowFile> flowFiles = controller.getFlowFilesForRelationship(ExecuteStreamCommand.OUTPUT_STREAM_RELATIONSHIP);
         MockFlowFile flowFile = flowFiles.get(0);
         assertEquals(0, flowFile.getSize());
-        assertEquals("fffffffffffffffffffffffffffffff", flowFile.getAttribute("execution.error").substring(0, 31));
+        assertTrue(flowFile.getAttribute("execution.error").contains("fffffffffffffffffffffffffffffff"));
     }
 
     @Test


[nifi] 05/16: NIFI-7766 Getting initialize class org.apache.nifi.util.text.RegexDateTimeMatcher error in jsontreereader

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.12.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 996c16947ab5bc7d4f415178dc4d44fd2915046a
Author: Otto Fowler <ot...@gmail.com>
AuthorDate: Tue Aug 25 14:36:55 2020 -0400

    NIFI-7766 Getting initialize class org.apache.nifi.util.text.RegexDateTimeMatcher error in jsontreereader
    
    check for null values returned by getZoneStrings()
---
 .../main/java/org/apache/nifi/util/text/RegexDateTimeMatcher.java  | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)

diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/RegexDateTimeMatcher.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/RegexDateTimeMatcher.java
index fe1b919..48f0774 100644
--- a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/RegexDateTimeMatcher.java
+++ b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/RegexDateTimeMatcher.java
@@ -131,9 +131,10 @@ public class RegexDateTimeMatcher implements DateTimeMatcher {
             final StringBuilder zoneNamePatternBuilder = new StringBuilder();
             for (final String[] zoneNames : zoneStrings) {
                 for (final String zoneName : zoneNames) {
-                    zoneNamePatternBuilder.append(Pattern.quote(zoneName)).append("|");
-
-                    maxTimeZoneLength = Math.max(maxTimeZoneLength, zoneName.length());
+                    if (zoneName != null && !zoneName.isEmpty()) {
+                        zoneNamePatternBuilder.append(Pattern.quote(zoneName)).append("|");
+                        maxTimeZoneLength = Math.max(maxTimeZoneLength, zoneName.length());
+                    }
                 }
             }
 


[nifi] 07/16: NIFI-7309 Update Admin Guide by removing unused properties and updating default values

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.12.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit a173426f503e2f5592c6697983850f31fd0c697a
Author: Andrew Lim <an...@gmail.com>
AuthorDate: Mon Aug 24 12:57:48 2020 -0400

    NIFI-7309 Update Admin Guide by removing unused properties and updating default values
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4488.
---
 .../src/main/asciidoc/administration-guide.adoc    | 23 ++++++----------------
 1 file changed, 6 insertions(+), 17 deletions(-)

diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc
index 752f5e6..ea2919f 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -1629,7 +1629,7 @@ image::zero-leader-cluster-http-access.png["NiFi Cluster HTTP Access"]
 
 === Zero-Leader Clustering
 NiFi employs a Zero-Leader Clustering paradigm. Each node in the cluster has an identical flow definition and performs the same tasks on
-the data, but each operates on a different set of data. The cluster automatically distributes the data throughout all the active nodes. 
+the data, but each operates on a different set of data. The cluster automatically distributes the data throughout all the active nodes.
 
 One of the nodes is automatically elected (via Apache
 ZooKeeper) as the Cluster Coordinator. All nodes in the cluster will then send heartbeat/status information
@@ -1765,7 +1765,7 @@ some amount of time has elapsed (configured by setting the `nifi.cluster.flow.el
 some number of Nodes have cast votes (configured by setting the `nifi.cluster.flow.election.max.candidates` property),
 a flow is elected to be the "correct" copy of the flow.
 
-Any node whose dataflow, users, groups, and policies conflict the with those elected will backup any conflicting resources and replace the local
+Any node whose dataflow, users, groups, and policies conflict with those elected will backup any conflicting resources and replace the local
 resources with those from the cluster. How the backup is performed depends on the configured Access Policy Provider and User Group Provider.
 For file-based access policy providers, the backup will be written to the same directory as the existing file (e.g., $NIFI_HOME/conf) and bear the same
 name but with a suffix of "." and a timestamp. For example, if the flow itself conflicts with the cluster's flow at 12:05:03 on January 1, 2020,
@@ -2596,7 +2596,6 @@ To do so, set the value of this property to `org.wali.MinimalLockingWriteAheadLo
 If the value of this property is changed, upon restart, NiFi will still recover the records written using the previously configured repository and delete the files written by the previously configured
 implementation.
 |`nifi.flowfile.repository.directory`*|The location of the FlowFile Repository. The default value is `./flowfile_repository`.
-|`nifi.flowfile.repository.partitions`|The number of partitions. The default value is `256`.
 |`nifi.flowfile.repository.checkpoint.interval`| The FlowFile Repository checkpoint interval. The default value is `2 mins`.
 |`nifi.flowfile.repository.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`.
 |====
@@ -2744,10 +2743,6 @@ available again. These properties govern how that process occurs.
 |*Property*|*Description*
 |`nifi.swap.manager.implementation`|The Swap Manager implementation. The default value is `org.apache.nifi.controller.FileSystemSwapManager` and should not be changed.
 |`nifi.queue.swap.threshold`|The queue threshold at which NiFi starts to swap FlowFile information to disk. The default value is `20000`.
-|`nifi.swap.in.period`|The swap in period. The default value is `5 sec`.
-|`nifi.swap.in.threads`|The number of threads to use for swapping in. The default value is `1`.
-|`nifi.swap.out.period`|The swap out period. The default value is `5 sec`.
-|`nifi.swap.out.threads`|The number of threads to use for swapping out. The default value is `4`.
 |====
 
 === Content Repository
@@ -2769,7 +2764,6 @@ FlowFile Repository, if also on that disk, could become corrupt. To avoid this s
 |*Property*|*Description*
 |`nifi.content.repository.implementation`|The Content Repository implementation. The default value is `org.apache.nifi.controller.repository.FileSystemRepository` and should only be changed with caution. To store flowfile content in memory instead of on disk (at the risk of data loss in the event of power/machine failure), set this property to `org.apache.nifi.controller.repository.VolatileContentRepository`.
 |`nifi.content.claim.max.appendable.size`|The maximum size for a content claim. The default value is `1 MB`.
-|`nifi.content.claim.max.flow.files`|The maximum number of FlowFiles to assign to one content claim. The default value is `100`.
 |`nifi.content.repository.directory.default`*|The location of the Content Repository. The default value is `./content_repository`. +
  +
 *NOTE*: Multiple content repositories can be specified by using the `nifi.content.repository.directory.` prefix with unique suffixes and separate paths as values. +
@@ -2868,12 +2862,9 @@ the `WriteAheadProvenanceRepository`, it cannot be changed back to the `Persiste
 	Providing three total locations, including `nifi.provenance.repository.directory.default`.
 |`nifi.provenance.repository.max.storage.time`|The maximum amount of time to keep data provenance information. The default value is `24 hours`.
 |`nifi.provenance.repository.max.storage.size`|The maximum amount of data provenance information to store at a time.
-	The default value is `1 GB`. The Data Provenance capability can consume a great deal of storage space because so much data is kept.
+	The default value is `10 GB`. The Data Provenance capability can consume a great deal of storage space because so much data is kept.
 	For production environments, values of 1-2 TB or more is not uncommon. The repository will write to a single "event file" (or set of
-	"event files" if multiple storage locations are defined, as described above) for some period of time (defined by the
-	`nifi.provenance.repository.rollover.time` and `nifi.provenance.repository.rollover.size` properties). Data is always aged off one file at a time,
-	so it is not advisable to write to a single "event file" for a tremendous amount of time, as it will prevent old data from aging off as smoothly.
-|`nifi.provenance.repository.rollover.time`|The amount of time to wait before rolling over the "event file" that the repository is writing to.
+	"event files" if multiple storage locations are defined, as described above) until the event file reaches the size defined in the `nifi.provenance.repository.rollover.size` property. It will then "roll over" and begin writing new events to a new file. Data is always aged off one file at a time, so it is not advisable to write a tremendous amount of data to a single "event file," as it will prevent old data from aging off as smoothly.
 |`nifi.provenance.repository.rollover.size`|The amount of data to write to a single "event file." The default value is `100 MB`. For production
 	environments where a very large amount of Data Provenance is generated, a value of `1 GB` is also very reasonable.
 |`nifi.provenance.repository.query.threads`|The number of threads to use for Provenance Repository queries. The default value is `2`.
@@ -2924,7 +2915,6 @@ All of the properties defined above (see <<write-ahead-provenance-repository-pro
 
 |====
 |*Property*|*Description*
-|`nifi.provenance.repository.debug.frequency`|Controls the number of events processed between DEBUG statements documenting the performance metrics of the repository. This value is only used when DEBUG level statements are enabled in the log configuration.
  |`nifi.provenance.repository.encryption.key.provider.implementation`|This is the fully-qualified class name of the **key provider**. A key provider is the datastore interface for accessing the encryption key to protect the provenance events. There are currently two implementations -- `StaticKeyProvider` which reads a key directly from _nifi.properties_, and `FileBasedKeyProvider` which reads *n* many keys from an encrypted file. The interface is extensible, and HSM-backed or other provi [...]
  |`nifi.provenance.repository.encryption.key.provider.location`|The path to the key definition resource (empty for `StaticKeyProvider`, `./keys.nkp` or similar path for `FileBasedKeyProvider`). For future providers like an HSM, this may be a connection string or URL.
  |`nifi.provenance.repository.encryption.key.id`|The active key ID to use for encryption (e.g. `Key1`).
@@ -2936,7 +2926,6 @@ The simplest configuration is below:
 
 ....
 nifi.provenance.repository.implementation=org.apache.nifi.provenance.EncryptedWriteAheadProvenanceRepository
-nifi.provenance.repository.debug.frequency=100
 nifi.provenance.repository.encryption.key.provider.implementation=org.apache.nifi.security.kms.StaticKeyProvider
 nifi.provenance.repository.encryption.key.provider.location=
 nifi.provenance.repository.encryption.key.id=Key1
@@ -2959,7 +2948,7 @@ For example, to provide two additional locations to act as part of the provenanc
  +
 Providing three total locations, including `nifi.provenance.repository.directory.default`.
 |`nifi.provenance.repository.max.storage.time`|The maximum amount of time to keep data provenance information. The default value is `24 hours`.
-|`nifi.provenance.repository.max.storage.size`|The maximum amount of data provenance information to store at a time. The default value is `1 GB`.
+|`nifi.provenance.repository.max.storage.size`|The maximum amount of data provenance information to store at a time. The default value is `10 GB`.
 |`nifi.provenance.repository.rollover.time`|The amount of time to wait before rolling over the latest data provenance information so that it is available in the User Interface. The default value is `30 secs`.
 |`nifi.provenance.repository.rollover.size`|The amount of information to roll over at a time. The default value is `100 MB`.
 |`nifi.provenance.repository.query.threads`|The number of threads to use for Provenance Repository queries. The default value is `2`.
@@ -3418,7 +3407,7 @@ to the cluster. It provides an additional layer of security. This value is blank
 long time before starting processing if we reach at least this number of nodes in the cluster.
 |`nifi.cluster.load.balance.port`|Specifies the port to listen on for incoming connections for load balancing data across the cluster. The default value is `6342`.
 |`nifi.cluster.load.balance.host`|Specifies the hostname to listen on for incoming connections for load balancing data across the cluster. If not specified, will default to the value used by the `nifi.cluster.node.address` property.
-|`nifi.cluster.load.balance.connections.per.node`|The maximum number of connections to create between this node and each other node in the cluster. For example, if there are 5 nodes in the cluster and this value is set to 4, there will be up to 20 socket connections established for load-balancing purposes (5 x 4 = 20). The default value is `4`.
+|`nifi.cluster.load.balance.connections.per.node`|The maximum number of connections to create between this node and each other node in the cluster. For example, if there are 5 nodes in the cluster and this value is set to 4, there will be up to 20 socket connections established for load-balancing purposes (5 x 4 = 20). The default value is `1`.
 |`nifi.cluster.load.balance.max.thread.count`|The maximum number of threads to use for transferring data from this node to other nodes in the cluster. While a given thread can only write to a single socket at a time, a single thread is capable of servicing multiple connections simultaneously because a given connection may not be available for reading/writing at any given time. The default value is `8`—i.e., up to 8 threads will be responsible for transferring data to other nodes, regardl [...]
 
 *NOTE:* Increasing this value will allow additional threads to be used for communicating with other nodes in the cluster and writing the data to the Content and FlowFile Repositories. However, if this property is set to a value greater than the number of nodes in the cluster multiplied by the number of connections per node (`nifi.cluster.load.balance.connections.per.node`), then no further benefit will be gained and resources will be wasted.


[nifi] 11/16: NIFI-7779: Avoid NPE when a ExecuteScript Processor has null value properties during a component search from canvas

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.12.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit e061964d9405e64943a92bfccd40f6f08aaf13a1
Author: Mohammed Nadeem <na...@gmail.com>
AuthorDate: Tue Sep 1 12:51:35 2020 +0530

    NIFI-7779: Avoid NPE when a ExecuteScript Processor has null value properties during a component search from canvas
---
 .../src/main/java/org/apache/nifi/script/ScriptingComponentUtils.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentUtils.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentUtils.java
index a889f4b..ea9ae26 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentUtils.java
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentUtils.java
@@ -89,7 +89,9 @@ public class ScriptingComponentUtils {
         final String scriptFile = context.getProperty(ScriptingComponentUtils.SCRIPT_FILE).evaluateAttributeExpressions().getValue();
         String script = context.getProperty(ScriptingComponentUtils.SCRIPT_BODY).getValue();
 
-        if (StringUtils.isBlank(script)) {
+        if (StringUtils.isBlank(script) && StringUtils.isBlank(scriptFile)) {
+            return results;
+        } else if (StringUtils.isBlank(script)) {
             try {
                 script = IOUtils.toString(new FileInputStream(scriptFile), StandardCharsets.UTF_8);
             } catch (Exception e) {


[nifi] 13/16: NIFI-7778: Made corrections in descriptions of padLeft, padRight, plus (#4504)

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.12.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 5da33ee909ac8637ed550bea0f6e690f19e142a5
Author: VKadam <56...@users.noreply.github.com>
AuthorDate: Tue Sep 1 13:07:47 2020 -0700

    NIFI-7778: Made corrections in descriptions of padLeft, padRight, plus (#4504)
    
    Signed-off-by: Andy LoPresto <al...@apache.org>
---
 nifi-docs/src/main/asciidoc/expression-language-guide.adoc | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/nifi-docs/src/main/asciidoc/expression-language-guide.adoc b/nifi-docs/src/main/asciidoc/expression-language-guide.adoc
index 354d1c2..aad0db2 100644
--- a/nifi-docs/src/main/asciidoc/expression-language-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/expression-language-guide.adoc
@@ -980,7 +980,7 @@ Expressions will provide the following results:
 [.function]
 === padLeft
 
-*Description*: [.description]#The `padLeft` function prepends the given padding string (or `'_'`, if nothing is provided) to the argument `String` until the passed desired length is reached.
+*Description*: [.description]#The `padLeft` function prepends the given padding string (or `'_'`, if nothing is provided) to the argument `String` until the passed desired length is reached.#
 
 It returns the argument as is if its length is already equal or higher than the desired length, if the padding string is `null`, and if the desired length is either negative or greater than `Integer.MAX_VALUE`.
 It returns `null` if the argument string is not a valid attribute.
@@ -1015,7 +1015,7 @@ Expressions will provide the following results:
 [.function]
 === padRight
 
-*Description*: [.description]#The `padRight` function appends the given padding string (or `'_'`, if nothing is provided) to the argument `String` until the passed desired length is reached.
+*Description*: [.description]#The `padRight` function appends the given padding string (or `'_'`, if nothing is provided) to the argument `String` until the passed desired length is reached.#
 
 It returns the argument as is if its length is already equal or higher than the desired length, if the padding string is `null`, and if the desired length is either negative or greater than `Integer.MAX_VALUE`.
 It returns `null` if the argument string is not a valid attribute.
@@ -1981,7 +1981,7 @@ Divide. This is to preserve backwards compatibility and to not force rounding er
 === plus
 
 *Description*: [.description]#Adds a numeric value to the Subject. If either the argument or the Subject cannot be
-	coerced into a Number, returns `null`. Does not provide handling for overflow.
+	coerced into a Number, returns `null`. Does not provide handling for overflow.#
 
 *Subject Type*: [.subject]#Number or Decimal#
 


[nifi] 02/16: NIFI-7758: Avoid calling InetAddress.getHostName() because doing so results in a reverse DNS Lookup, which can be expensive

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.12.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 669054c4686da6df62a23904c8dfea1186d1c05d
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Mon Aug 24 09:11:00 2020 -0400

    NIFI-7758: Avoid calling InetAddress.getHostName() because doing so results in a reverse DNS Lookup, which can be expensive
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4487.
---
 .../apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)

diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
index dc36926..6dd167f 100644
--- a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
@@ -51,7 +51,7 @@ public class SSLSocketChannel implements Closeable {
     private static final Logger logger = LoggerFactory.getLogger(SSLSocketChannel.class);
     private static final long BUFFER_FULL_EMPTY_WAIT_NANOS = TimeUnit.NANOSECONDS.convert(1, TimeUnit.MILLISECONDS);
 
-    private final String hostname;
+    private final String remoteAddress;
     private final int port;
     private final SSLEngine engine;
     private final SocketAddress socketAddress;
@@ -77,7 +77,7 @@ public class SSLSocketChannel implements Closeable {
             final SocketAddress localSocketAddress = new InetSocketAddress(localAddress, 0);
             this.channel.bind(localSocketAddress);
         }
-        this.hostname = hostname;
+        this.remoteAddress = hostname;
         this.port = port;
         this.engine = sslContext.createSSLEngine();
         this.engine.setUseClientMode(client);
@@ -97,7 +97,7 @@ public class SSLSocketChannel implements Closeable {
 
         this.socketAddress = socketChannel.getRemoteAddress();
         final Socket socket = socketChannel.socket();
-        this.hostname = socket.getInetAddress().getHostName();
+        this.remoteAddress = socket.getInetAddress().toString();
         this.port = socket.getPort();
 
         this.engine = sslContext.createSSLEngine();
@@ -118,7 +118,7 @@ public class SSLSocketChannel implements Closeable {
 
         this.socketAddress = socketChannel.getRemoteAddress();
         final Socket socket = socketChannel.socket();
-        this.hostname = socket.getInetAddress().getHostName();
+        this.remoteAddress = socket.getInetAddress().toString();
         this.port = socket.getPort();
 
         // don't set useClientMode or needClientAuth, use the engine as is and let the caller configure it
@@ -149,7 +149,7 @@ public class SSLSocketChannel implements Closeable {
                             throw new TransmissionDisabledException();
                         }
                         if (System.currentTimeMillis() > startTime + timeoutMillis) {
-                            throw new SocketTimeoutException("Timed out connecting to " + hostname + ":" + port);
+                            throw new SocketTimeoutException("Timed out connecting to " + remoteAddress + ":" + port);
                         }
 
                         try {
@@ -311,7 +311,7 @@ public class SSLSocketChannel implements Closeable {
             long sleepNanos = 1L;
             if (readCount == 0) {
                 if (System.currentTimeMillis() > startTime + timeoutMillis) {
-                    throw new SocketTimeoutException("Timed out reading from socket connected to " + hostname + ":" + port);
+                    throw new SocketTimeoutException("Timed out reading from socket connected to " + remoteAddress + ":" + port);
                 }
                 try {
                     TimeUnit.NANOSECONDS.sleep(sleepNanos);
@@ -370,7 +370,7 @@ public class SSLSocketChannel implements Closeable {
                 lastByteWrittenTime = now;
             } else {
                 if (now > lastByteWrittenTime + timeoutMillis) {
-                    throw new SocketTimeoutException("Timed out writing to socket connected to " + hostname + ":" + port);
+                    throw new SocketTimeoutException("Timed out writing to socket connected to " + remoteAddress + ":" + port);
                 }
                 try {
                     TimeUnit.NANOSECONDS.sleep(sleepNanos);


[nifi] 04/16: NIFI-7760 Remove invalid permission check

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.12.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 6db63dea5a1dc175a9b42ed9737f0fa979b68ba2
Author: Tamás Bunth <bt...@gmail.com>
AuthorDate: Tue Aug 25 13:31:58 2020 +0200

    NIFI-7760 Remove invalid permission check
    
    Because even though permissions are only the 7 least significant bits of the file
    mode but the file mode can be wider and can contain further info (like the
    sticky bit).
    
    Extend unit test for converting file mode with sticky bit into 'rwx' style
    permission string.
    
    Remove old test cases
    
    This closes #4490.
    
    Signed-off-by: Peter Turcsanyi <tu...@apache.org>
---
 .../org/apache/nifi/processors/standard/util/FileInfo.java |  4 ----
 .../apache/nifi/processors/standard/util/TestFileInfo.java | 14 ++++----------
 2 files changed, 4 insertions(+), 14 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileInfo.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileInfo.java
index 763ad07..20efdf5 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileInfo.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileInfo.java
@@ -220,10 +220,6 @@ public class FileInfo implements Comparable<FileInfo>, Serializable, ListableEnt
     }
 
     public static String permissionToString(int fileModeOctal) {
-        if (fileModeOctal > 0777 || fileModeOctal < 00) {
-            throw new IllegalArgumentException("Invalid permission numerals");
-        }
-
         StringBuilder sb = new StringBuilder();
         for (char p : PERMISSION_MODIFIER_CHARS) {
             sb.append((fileModeOctal & 1) == 1 ? p : '-');
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestFileInfo.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestFileInfo.java
index 1ff9bd3..afaf108 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestFileInfo.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestFileInfo.java
@@ -27,19 +27,13 @@ public class TestFileInfo {
         String rwxPerm = FileInfo.permissionToString(0567);
         assertEquals("r-xrw-rwx", rwxPerm);
 
+        // Test with sticky bit
+        rwxPerm = FileInfo.permissionToString(01567);
+        assertEquals("r-xrw-rwx", rwxPerm);
+
         rwxPerm = FileInfo.permissionToString(03);
         assertEquals("-------wx", rwxPerm);
 
     }
 
-    @Test(expected = IllegalArgumentException.class)
-    public void testPermissionModeToStringInvalidFourDigits() {
-        FileInfo.permissionToString(01000);
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testPermissionModeToStringInvalidNegative() {
-        FileInfo.permissionToString(-1);
-    }
-
 }


[nifi] 09/16: NIFI-7661 Change validation of controller service against it's API to use comparison of inteface methods instead of bundle coordinates

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.12.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit f4bce4fead6adf78a8caa00bfcc2b5f71a26cfd0
Author: Bryan Bende <bb...@apache.org>
AuthorDate: Fri Jul 17 16:19:02 2020 -0400

    NIFI-7661 Change validation of controller service against it's API to use comparison of inteface methods instead of bundle coordinates
---
 .../nifi/controller/AbstractComponentNode.java     |  33 ++-
 .../controller/ControllerServiceApiMatcher.java    |  73 ++++++
 .../TestControllerServiceApiMatcher.java           | 283 +++++++++++++++++++++
 3 files changed, 377 insertions(+), 12 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
index a7db9ec..0643418 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
@@ -727,6 +727,7 @@ public abstract class AbstractComponentNode implements ComponentNode {
         if (controllerServiceApiClass.equals(ControllerService.class)) {
             return null;
         }
+
         final ClassLoader controllerServiceApiClassLoader = controllerServiceApiClass.getClassLoader();
         final ExtensionManager extensionManager = serviceProvider.getExtensionManager();
 
@@ -745,20 +746,28 @@ public abstract class AbstractComponentNode implements ComponentNode {
         }
         final BundleCoordinate controllerServiceCoordinate = controllerServiceBundle.getBundleDetails().getCoordinate();
 
-        final boolean matchesApi = matchesApi(extensionManager, controllerServiceBundle, controllerServiceApiCoordinate);
+        final boolean matchesApiByBundleCoordinates = matchesApiBundleCoordinates(extensionManager, controllerServiceBundle, controllerServiceApiCoordinate);
+        if (!matchesApiByBundleCoordinates) {
+            final Class<? extends ControllerService> controllerServiceImplClass = controllerServiceNode.getControllerServiceImplementation().getClass();
+            logger.debug("Comparing methods from service api '{}' against service implementation '{}'",
+                    new Object[]{controllerServiceApiClass.getCanonicalName(), controllerServiceImplClass.getCanonicalName()});
+
+            final ControllerServiceApiMatcher controllerServiceApiMatcher = new ControllerServiceApiMatcher();
+            final boolean matchesApi = controllerServiceApiMatcher.matches(controllerServiceApiClass, controllerServiceImplClass);
 
-        if (!matchesApi) {
-            final String controllerServiceType = controllerServiceNode.getComponentType();
-            final String controllerServiceApiType = controllerServiceApiClass.getSimpleName();
+            if (!matchesApi) {
+                final String controllerServiceType = controllerServiceNode.getComponentType();
+                final String controllerServiceApiType = controllerServiceApiClass.getSimpleName();
 
-            final String explanation = new StringBuilder()
-                .append(controllerServiceType).append(" - ").append(controllerServiceCoordinate.getVersion())
-                .append(" from ").append(controllerServiceCoordinate.getGroup()).append(" - ").append(controllerServiceCoordinate.getId())
-                .append(" is not compatible with ").append(controllerServiceApiType).append(" - ").append(controllerServiceApiCoordinate.getVersion())
-                .append(" from ").append(controllerServiceApiCoordinate.getGroup()).append(" - ").append(controllerServiceApiCoordinate.getId())
-                .toString();
+                final String explanation = new StringBuilder()
+                        .append(controllerServiceType).append(" - ").append(controllerServiceCoordinate.getVersion())
+                        .append(" from ").append(controllerServiceCoordinate.getGroup()).append(" - ").append(controllerServiceCoordinate.getId())
+                        .append(" is not compatible with ").append(controllerServiceApiType).append(" - ").append(controllerServiceApiCoordinate.getVersion())
+                        .append(" from ").append(controllerServiceApiCoordinate.getGroup()).append(" - ").append(controllerServiceApiCoordinate.getId())
+                        .toString();
 
-            return createInvalidResult(serviceId, propertyName, explanation);
+                return createInvalidResult(serviceId, propertyName, explanation);
+            }
         }
 
         return null;
@@ -780,7 +789,7 @@ public abstract class AbstractComponentNode implements ComponentNode {
      * @param requiredApiCoordinate the controller service API required by the processor
      * @return true if the controller service node has the require API as an ancestor, false otherwise
      */
-    private boolean matchesApi(final ExtensionManager extensionManager, final Bundle controllerServiceImplBundle, final BundleCoordinate requiredApiCoordinate) {
+    private boolean matchesApiBundleCoordinates(final ExtensionManager extensionManager, final Bundle controllerServiceImplBundle, final BundleCoordinate requiredApiCoordinate) {
         // start with the coordinate of the controller service for cases where the API and service are in the same bundle
         BundleCoordinate controllerServiceDependencyCoordinate = controllerServiceImplBundle.getBundleDetails().getCoordinate();
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ControllerServiceApiMatcher.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ControllerServiceApiMatcher.java
new file mode 100644
index 0000000..5183617
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ControllerServiceApiMatcher.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.controller;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.Arrays;
+
+public class ControllerServiceApiMatcher {
+
+    private static Logger LOGGER = LoggerFactory.getLogger(ControllerServiceApiMatcher.class);
+
+    /**
+     * Determines if all of the methods from the API class are present in the implementation class.
+     *
+     * @param serviceApi the controller service API class
+     * @param serviceImplementation the controller service implementation class
+     * @return true if all API methods exists in the implementation with the same name, parameters, and return type
+     */
+    public boolean matches(Class<? extends ControllerService> serviceApi, Class<? extends ControllerService> serviceImplementation) {
+        for (final Method apiMethod : serviceApi.getMethods()) {
+            boolean foundMatchingImplMethod = false;
+            for (final Method implMethod : serviceImplementation.getMethods()) {
+                if (!apiMethod.getName().equals(implMethod.getName())) {
+                    continue;
+                }
+
+                // if the service interface has new methods that the implementation doesn't implement,
+                // those methods still show up list of methods when calling impl getMethods(), but they
+                // are marked as abstract and will produce an AbstractMethodError at runtime if invoked
+                if (Modifier.isAbstract(implMethod.getModifiers())) {
+                    continue;
+                }
+
+                final boolean returnTypeMatches = apiMethod.getReturnType().equals(implMethod.getReturnType());
+                final boolean argsMatch = Arrays.equals(apiMethod.getParameterTypes(), implMethod.getParameterTypes());
+
+                if (returnTypeMatches && argsMatch) {
+                    foundMatchingImplMethod = true;
+                    break;
+                }
+            }
+
+            if (!foundMatchingImplMethod) {
+                if (LOGGER.isDebugEnabled()) {
+                    LOGGER.debug("{} does not implement the API method [{}] from {}",
+                            new Object[]{serviceImplementation.getCanonicalName(), apiMethod.toString(), serviceApi.getCanonicalName()});
+                }
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestControllerServiceApiMatcher.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestControllerServiceApiMatcher.java
new file mode 100644
index 0000000..4bc1849
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestControllerServiceApiMatcher.java
@@ -0,0 +1,283 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.controller;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.reporting.InitializationException;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.List;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestControllerServiceApiMatcher {
+
+    private ControllerServiceApiMatcher serviceApiMatcher;
+
+    @Before
+    public void setup() {
+        this.serviceApiMatcher = new ControllerServiceApiMatcher();
+    }
+
+    @Test
+    public void testServiceImplementationMatchesRealServiceApi() {
+        assertTrue(serviceApiMatcher.matches(FooServiceApiV1.class, FooServiceImplementationV1.class));
+    }
+
+    @Test
+    public void testServiceImplementationMatchesCompatibleServiceApi() {
+        // v2 implementation should match the real v2 API that it implements
+        assertTrue(serviceApiMatcher.matches(FooServiceApiV2.class, FooServiceImplementationV2.class));
+        // v2 implementation should also match the v1 API since the methods are the same
+        assertTrue(serviceApiMatcher.matches(FooServiceApiV1.class, FooServiceImplementationV2.class));
+    }
+
+    @Test
+    public void testServiceImplementationDoesNotMatchIncompatibleApi() {
+        assertFalse(serviceApiMatcher.matches(FooServiceApiV3.class, FooServiceImplementationV1.class));
+        assertFalse(serviceApiMatcher.matches(FooServiceApiV3.class, FooServiceImplementationV2.class));
+    }
+
+    @Test
+    public void testServiceApiWithGenerics() {
+        // should match
+        assertTrue(serviceApiMatcher.matches(GenericServiceApiV1.class, GenericServiceImpl.class));
+        // should not match because method changed args
+        assertFalse(serviceApiMatcher.matches(GenericServiceApiV2.class, GenericServiceImpl.class));
+        // should not match because method changed return type
+        assertFalse(serviceApiMatcher.matches(GenericServiceApiV3.class, GenericServiceImpl.class));
+    }
+
+    // Interface for a result
+    private interface FooResult {
+        String getResult();
+    }
+
+    // Implementation for a result
+    private class FooResultImpl implements FooResult {
+        private final String result;
+
+        public FooResultImpl(final String result) {
+            this.result = result;
+        }
+
+        @Override
+        public String getResult() {
+            return result;
+        }
+    }
+
+    // Interface for an argument
+    private interface FooArg {
+        String getArg();
+    }
+
+    // Implementation for an argument
+    private static class FooArgImpl implements FooArg {
+        private final String arg;
+
+        public FooArgImpl(final String arg) {
+            this.arg = arg;
+        }
+
+        @Override
+        public String getArg() {
+            return arg;
+        }
+    }
+
+    // Service API v1
+    private interface FooServiceApiV1 extends ControllerService {
+        void execute();
+        String execute(String a);
+        FooResult executeWithReturn(FooArg a);
+    }
+
+    // Service API v2, unchanged from v1
+    private interface FooServiceApiV2 extends ControllerService {
+        void execute();
+        String execute(String a);
+        FooResult executeWithReturn(FooArg a);
+    }
+
+    // Service API v3, new method added since v1/v3
+    private interface FooServiceApiV3 extends ControllerService {
+        void execute();
+        String execute(String a);
+        FooResult executeWithReturn(FooArg a);
+        String someNewMethod();
+    }
+
+    // Service implementing v1 API
+    private static class FooServiceImplementationV1 implements FooServiceApiV1 {
+
+        @Override
+        public void execute() {
+
+        }
+
+        @Override
+        public String execute(String a) {
+            return null;
+        }
+
+        // Declare the impl result here to test comparing a more specific return type against the API
+        @Override
+        public FooResultImpl executeWithReturn(FooArg a) {
+            return null;
+        }
+
+        @Override
+        public void initialize(ControllerServiceInitializationContext context) throws InitializationException {
+
+        }
+
+        @Override
+        public Collection<ValidationResult> validate(ValidationContext context) {
+            return null;
+        }
+
+        @Override
+        public PropertyDescriptor getPropertyDescriptor(String name) {
+            return null;
+        }
+
+        @Override
+        public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+
+        }
+
+        @Override
+        public List<PropertyDescriptor> getPropertyDescriptors() {
+            return null;
+        }
+
+        @Override
+        public String getIdentifier() {
+            return null;
+        }
+    }
+
+    // Service implementing v2 API
+    private static class FooServiceImplementationV2 implements FooServiceApiV2 {
+
+        @Override
+        public void execute() {
+
+        }
+
+        @Override
+        public String execute(String a) {
+            return null;
+        }
+
+        @Override
+        public FooResult executeWithReturn(FooArg a) {
+            return null;
+        }
+
+        @Override
+        public void initialize(ControllerServiceInitializationContext context) throws InitializationException {
+
+        }
+
+        @Override
+        public Collection<ValidationResult> validate(ValidationContext context) {
+            return null;
+        }
+
+        @Override
+        public PropertyDescriptor getPropertyDescriptor(String name) {
+            return null;
+        }
+
+        @Override
+        public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+
+        }
+
+        @Override
+        public List<PropertyDescriptor> getPropertyDescriptors() {
+            return null;
+        }
+
+        @Override
+        public String getIdentifier() {
+            return null;
+        }
+    }
+
+    // Service API with generics v1
+    private interface GenericServiceApiV1<T extends FooResult, V extends FooArg> extends ControllerService {
+        T execute(V arg);
+    }
+
+    // Service API with generics v2
+    private interface GenericServiceApiV2<T extends FooResult, V extends FooArg> extends ControllerService {
+        T execute(V arg1, V arg2);
+    }
+
+    // Service API with generics v3
+    private interface GenericServiceApiV3<V extends FooArg> extends ControllerService {
+        String execute(V arg);
+    }
+
+    // Service implementation with generics
+    private static class GenericServiceImpl implements GenericServiceApiV1<FooResultImpl,FooArgImpl> {
+
+        @Override
+        public FooResultImpl execute(FooArgImpl arg) {
+            return null;
+        }
+
+        @Override
+        public void initialize(ControllerServiceInitializationContext context) throws InitializationException {
+
+        }
+
+        @Override
+        public Collection<ValidationResult> validate(ValidationContext context) {
+            return null;
+        }
+
+        @Override
+        public PropertyDescriptor getPropertyDescriptor(String name) {
+            return null;
+        }
+
+        @Override
+        public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+
+        }
+
+        @Override
+        public List<PropertyDescriptor> getPropertyDescriptors() {
+            return null;
+        }
+
+        @Override
+        public String getIdentifier() {
+            return null;
+        }
+    }
+
+}


[nifi] 01/16: NIFI-7787 prepping versions for support/1.12.x branch starting at 1.12.1

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.12.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 6c2f59f50e791ac4975cee38ee036474be160317
Author: Joe Witt <jo...@apache.org>
AuthorDate: Wed Sep 2 11:52:33 2020 -0500

    NIFI-7787 prepping versions for support/1.12.x branch starting at 1.12.1
---
 nifi-api/pom.xml                                   |   2 +-
 nifi-assembly/pom.xml                              | 258 ++++++++++-----------
 nifi-bootstrap/pom.xml                             |  12 +-
 nifi-commons/nifi-data-provenance-utils/pom.xml    |  14 +-
 nifi-commons/nifi-expression-language/pom.xml      |   8 +-
 nifi-commons/nifi-flowfile-packager/pom.xml        |   2 +-
 nifi-commons/nifi-hl7-query-language/pom.xml       |   2 +-
 nifi-commons/nifi-json-utils/pom.xml               |   6 +-
 nifi-commons/nifi-logging-utils/pom.xml            |   2 +-
 nifi-commons/nifi-metrics/pom.xml                  |   4 +-
 nifi-commons/nifi-parameter/pom.xml                |   4 +-
 nifi-commons/nifi-properties/pom.xml               |   2 +-
 nifi-commons/nifi-record-path/pom.xml              |   8 +-
 nifi-commons/nifi-record/pom.xml                   |   2 +-
 nifi-commons/nifi-rocksdb-utils/pom.xml            |   8 +-
 nifi-commons/nifi-schema-utils/pom.xml             |   2 +-
 nifi-commons/nifi-security-utils/pom.xml           |   8 +-
 nifi-commons/nifi-site-to-site-client/pom.xml      |  12 +-
 nifi-commons/nifi-socket-utils/pom.xml             |   8 +-
 nifi-commons/nifi-utils/pom.xml                    |   6 +-
 nifi-commons/nifi-web-utils/pom.xml                |   4 +-
 nifi-commons/nifi-write-ahead-log/pom.xml          |   4 +-
 nifi-commons/pom.xml                               |   2 +-
 nifi-docker/docker-compose/docker-compose.yml      |   2 +-
 nifi-docker/dockerhub/DockerImage.txt              |   2 +-
 nifi-docker/dockerhub/Dockerfile                   |   2 +-
 nifi-docker/dockerhub/README.md                    |   4 +-
 nifi-docker/dockerhub/pom.xml                      |   2 +-
 nifi-docker/dockermaven-stateless/pom.xml          |   2 +-
 nifi-docker/dockermaven/pom.xml                    |   2 +-
 nifi-docker/pom.xml                                |   4 +-
 nifi-docs/pom.xml                                  |   2 +-
 .../nifi-nifi-example-nar/pom.xml                  |   2 +-
 .../nifi-nifi-example-processors/pom.xml           |   8 +-
 nifi-external/nifi-example-bundle/pom.xml          |   4 +-
 nifi-external/nifi-spark-receiver/pom.xml          |   4 +-
 nifi-external/nifi-storm-spout/pom.xml             |   4 +-
 nifi-external/pom.xml                              |   2 +-
 nifi-framework-api/pom.xml                         |   6 +-
 .../nifi-processor-bundle-archetype/pom.xml        |   2 +-
 .../nifi-service-bundle-archetype/pom.xml          |   2 +-
 nifi-maven-archetypes/pom.xml                      |   2 +-
 nifi-mock/pom.xml                                  |  14 +-
 .../nifi-accumulo-bundle/nifi-accumulo-nar/pom.xml |  10 +-
 .../nifi-accumulo-processors/pom.xml               |  18 +-
 .../nifi-accumulo-services-api-nar/pom.xml         |  10 +-
 .../nifi-accumulo-services-api/pom.xml             |  10 +-
 .../nifi-accumulo-services-nar/pom.xml             |  10 +-
 .../nifi-accumulo-services/pom.xml                 |  14 +-
 nifi-nar-bundles/nifi-accumulo-bundle/pom.xml      |   8 +-
 .../nifi-ambari-bundle/nifi-ambari-nar/pom.xml     |   4 +-
 .../nifi-ambari-reporting-task/pom.xml             |  10 +-
 nifi-nar-bundles/nifi-ambari-bundle/pom.xml        |   2 +-
 .../nifi-amqp-bundle/nifi-amqp-nar/pom.xml         |   6 +-
 .../nifi-amqp-bundle/nifi-amqp-processors/pom.xml  |   6 +-
 nifi-nar-bundles/nifi-amqp-bundle/pom.xml          |   6 +-
 .../nifi-atlas-bundle/nifi-atlas-nar/pom.xml       |   4 +-
 .../nifi-atlas-reporting-task/pom.xml              |   8 +-
 nifi-nar-bundles/nifi-atlas-bundle/pom.xml         |   4 +-
 .../nifi-avro-bundle/nifi-avro-nar/pom.xml         |   6 +-
 .../nifi-avro-bundle/nifi-avro-processors/pom.xml  |   6 +-
 nifi-nar-bundles/nifi-avro-bundle/pom.xml          |   4 +-
 .../nifi-aws-abstract-processors/pom.xml           |   6 +-
 .../nifi-aws-bundle/nifi-aws-nar/pom.xml           |   6 +-
 .../nifi-aws-bundle/nifi-aws-processors/pom.xml    |  12 +-
 .../nifi-aws-service-api-nar/pom.xml               |   6 +-
 .../nifi-aws-bundle/nifi-aws-service-api/pom.xml   |   2 +-
 nifi-nar-bundles/nifi-aws-bundle/pom.xml           |   2 +-
 .../nifi-azure-bundle/nifi-azure-nar/pom.xml       |   8 +-
 .../nifi-azure-processors/pom.xml                  |  14 +-
 .../nifi-azure-reporting-task/pom.xml              |  12 +-
 .../nifi-azure-services-api-nar/pom.xml            |   6 +-
 .../nifi-azure-services-api/pom.xml                |   2 +-
 nifi-nar-bundles/nifi-azure-bundle/pom.xml         |   2 +-
 .../nifi-beats-bundle/nifi-beats-nar/pom.xml       |   8 +-
 .../nifi-beats-processors/pom.xml                  |  12 +-
 nifi-nar-bundles/nifi-beats-bundle/pom.xml         |   4 +-
 .../nifi-cassandra-nar/pom.xml                     |   4 +-
 .../nifi-cassandra-processors/pom.xml              |   6 +-
 .../nifi-cassandra-services-api-nar/pom.xml        |   8 +-
 .../nifi-cassandra-services-api/pom.xml            |   2 +-
 .../nifi-cassandra-services-nar/pom.xml            |   6 +-
 .../nifi-cassandra-services/pom.xml                |  14 +-
 nifi-nar-bundles/nifi-cassandra-bundle/pom.xml     |   4 +-
 .../nifi-ccda-bundle/nifi-ccda-nar/pom.xml         |   2 +-
 .../nifi-ccda-bundle/nifi-ccda-processors/pom.xml  |   6 +-
 nifi-nar-bundles/nifi-ccda-bundle/pom.xml          |   4 +-
 nifi-nar-bundles/nifi-cdc/nifi-cdc-api/pom.xml     |   2 +-
 .../nifi-cdc-mysql-nar/pom.xml                     |   4 +-
 .../nifi-cdc-mysql-processors/pom.xml              |  10 +-
 .../nifi-cdc/nifi-cdc-mysql-bundle/pom.xml         |   4 +-
 nifi-nar-bundles/nifi-cdc/pom.xml                  |   2 +-
 .../nifi-confluent-platform-nar/pom.xml            |   6 +-
 .../nifi-confluent-schema-registry-service/pom.xml |   6 +-
 .../nifi-confluent-platform-bundle/pom.xml         |   2 +-
 .../nifi-couchbase-nar/pom.xml                     |   8 +-
 .../nifi-couchbase-processors/pom.xml              |   8 +-
 .../nifi-couchbase-services-api-nar/pom.xml        |   8 +-
 .../nifi-couchbase-services-api/pom.xml            |   2 +-
 nifi-nar-bundles/nifi-couchbase-bundle/pom.xml     |   4 +-
 .../nifi-cybersecurity-nar/pom.xml                 |   6 +-
 .../nifi-cybersecurity-processors/pom.xml          |   6 +-
 nifi-nar-bundles/nifi-cybersecurity-bundle/pom.xml |   4 +-
 .../nifi-datadog-bundle/nifi-datadog-nar/pom.xml   |   2 +-
 .../nifi-datadog-reporting-task/pom.xml            |   8 +-
 nifi-nar-bundles/nifi-datadog-bundle/pom.xml       |   4 +-
 .../nifi-druid-controller-service-api-nar/pom.xml  |   6 +-
 .../nifi-druid-controller-service-api/pom.xml      |   2 +-
 .../nifi-druid-controller-service/pom.xml          |   8 +-
 .../nifi-druid-bundle/nifi-druid-nar/pom.xml       |   8 +-
 .../nifi-druid-processors/pom.xml                  |  12 +-
 nifi-nar-bundles/nifi-druid-bundle/pom.xml         |   4 +-
 .../nifi-easyrules-nar/pom.xml                     |   8 +-
 .../nifi-easyrules-service/pom.xml                 |   8 +-
 nifi-nar-bundles/nifi-easyrules-bundle/pom.xml     |   4 +-
 .../nifi-elasticsearch-5-nar/pom.xml               |   4 +-
 .../nifi-elasticsearch-5-processors/pom.xml        |   6 +-
 .../pom.xml                                        |   8 +-
 .../nifi-elasticsearch-client-service-api/pom.xml  |   8 +-
 .../nifi-elasticsearch-client-service-nar/pom.xml  |   8 +-
 .../nifi-elasticsearch-client-service/pom.xml      |  22 +-
 .../nifi-elasticsearch-nar/pom.xml                 |   4 +-
 .../nifi-elasticsearch-processors/pom.xml          |  14 +-
 .../nifi-elasticsearch-restapi-nar/pom.xml         |   6 +-
 .../nifi-elasticsearch-restapi-processors/pom.xml  |  26 +--
 nifi-nar-bundles/nifi-elasticsearch-bundle/pom.xml |   8 +-
 .../nifi-email-bundle/nifi-email-nar/pom.xml       |   4 +-
 .../nifi-email-processors/pom.xml                  |   6 +-
 nifi-nar-bundles/nifi-email-bundle/pom.xml         |   4 +-
 .../nifi-enrich-bundle/nifi-enrich-nar/pom.xml     |   2 +-
 .../nifi-enrich-processors/pom.xml                 |  12 +-
 nifi-nar-bundles/nifi-enrich-bundle/pom.xml        |   4 +-
 .../nifi-evtx-bundle/nifi-evtx-nar/pom.xml         |   4 +-
 .../nifi-evtx-bundle/nifi-evtx-processors/pom.xml  |   6 +-
 nifi-nar-bundles/nifi-evtx-bundle/pom.xml          |   4 +-
 .../nifi-database-test-utils/pom.xml               |   2 +-
 .../nifi-database-utils/pom.xml                    |   8 +-
 .../nifi-extension-utils/nifi-hadoop-utils/pom.xml |   8 +-
 .../nifi-processor-utils/pom.xml                   |   8 +-
 .../nifi-prometheus-utils/pom.xml                  |   8 +-
 .../nifi-avro-record-utils/pom.xml                 |   6 +-
 .../nifi-hadoop-record-utils/pom.xml               |   8 +-
 .../nifi-mock-record-utils/pom.xml                 |   6 +-
 .../nifi-standard-record-utils/pom.xml             |   8 +-
 .../nifi-extension-utils/nifi-record-utils/pom.xml |   2 +-
 .../nifi-reporting-utils/pom.xml                   |   8 +-
 .../nifi-service-utils/pom.xml                     |   6 +-
 .../nifi-extension-utils/nifi-syslog-utils/pom.xml |   6 +-
 nifi-nar-bundles/nifi-extension-utils/pom.xml      |   2 +-
 .../nifi-flume-bundle/nifi-flume-nar/pom.xml       |   6 +-
 .../nifi-flume-processors/pom.xml                  |   8 +-
 nifi-nar-bundles/nifi-flume-bundle/pom.xml         |   6 +-
 .../nifi-framework-nar/pom.xml                     |   6 +-
 .../nifi-framework/nifi-administration/pom.xml     |   2 +-
 .../nifi-framework/nifi-authorizer/pom.xml         |   4 +-
 .../nifi-framework/nifi-client-dto/pom.xml         |   2 +-
 .../nifi-framework/nifi-documentation/pom.xml      |   2 +-
 .../nifi-framework/nifi-file-authorizer/pom.xml    |   2 +-
 .../nifi-flowfile-repo-serialization/pom.xml       |   8 +-
 .../nifi-framework-authorization/pom.xml           |   6 +-
 .../nifi-framework-cluster-protocol/pom.xml        |   6 +-
 .../nifi-framework/nifi-framework-cluster/pom.xml  |   2 +-
 .../nifi-framework/nifi-framework-core-api/pom.xml |   6 +-
 .../nifi-framework/nifi-framework-core/pom.xml     |  28 +--
 .../nifi-framework-nar-loading-utils/pom.xml       |   2 +-
 .../nifi-framework-nar-utils/pom.xml               |   2 +-
 .../nifi-framework/nifi-mock-authorizer/pom.xml    |   2 +-
 .../nifi-framework/nifi-nar-utils/pom.xml          |   2 +-
 .../nifi-framework/nifi-properties-loader/pom.xml  |   2 +-
 .../nifi-framework/nifi-repository-models/pom.xml  |   6 +-
 .../nifi-framework/nifi-resources/pom.xml          |   2 +-
 .../nifi-framework/nifi-runtime/pom.xml            |   2 +-
 .../nifi-framework/nifi-shell-authorizer/pom.xml   |   4 +-
 .../nifi-framework/nifi-site-to-site/pom.xml       |  12 +-
 .../nifi-standard-prioritizers/pom.xml             |   2 +-
 .../nifi-framework/nifi-stateless/pom.xml          |  24 +-
 .../nifi-framework/nifi-user-actions/pom.xml       |   2 +-
 .../nifi-web/nifi-custom-ui-utilities/pom.xml      |   2 +-
 .../nifi-framework/nifi-web/nifi-jetty/pom.xml     |   2 +-
 .../nifi-web/nifi-ui-extension/pom.xml             |   2 +-
 .../nifi-framework/nifi-web/nifi-web-api/pom.xml   |   6 +-
 .../nifi-web/nifi-web-content-access/pom.xml       |   2 +-
 .../nifi-web/nifi-web-content-viewer/pom.xml       |   2 +-
 .../nifi-framework/nifi-web/nifi-web-docs/pom.xml  |   2 +-
 .../nifi-framework/nifi-web/nifi-web-error/pom.xml |   2 +-
 .../nifi-web/nifi-web-optimistic-locking/pom.xml   |   2 +-
 .../nifi-web/nifi-web-security/pom.xml             |   2 +-
 .../nifi-framework/nifi-web/nifi-web-ui/pom.xml    |   2 +-
 .../nifi-framework/nifi-web/pom.xml                |  12 +-
 .../nifi-framework-bundle/nifi-framework/pom.xml   |   2 +-
 nifi-nar-bundles/nifi-framework-bundle/pom.xml     |  68 +++---
 .../nifi-gcp-bundle/nifi-gcp-nar/pom.xml           |   6 +-
 .../nifi-gcp-bundle/nifi-gcp-processors/pom.xml    |  12 +-
 .../nifi-gcp-services-api-nar/pom.xml              |   8 +-
 .../nifi-gcp-bundle/nifi-gcp-services-api/pom.xml  |   2 +-
 nifi-nar-bundles/nifi-gcp-bundle/pom.xml           |   4 +-
 .../nifi-graph-client-service-api-nar/pom.xml      |   6 +-
 .../nifi-graph-client-service-api/pom.xml          |   4 +-
 .../nifi-graph-bundle/nifi-graph-nar/pom.xml       |   4 +-
 .../nifi-graph-processors/pom.xml                  |  10 +-
 .../nifi-graph-test-clients/pom.xml                |   6 +-
 .../nifi-neo4j-cypher-service-nar/pom.xml          |   6 +-
 .../nifi-neo4j-cypher-service/pom.xml              |  12 +-
 .../nifi-other-graph-services-nar/pom.xml          |   6 +-
 .../nifi-other-graph-services/pom.xml              |  12 +-
 nifi-nar-bundles/nifi-graph-bundle/pom.xml         |   4 +-
 .../nifi-groovyx-bundle/nifi-groovyx-nar/pom.xml   |   4 +-
 .../nifi-groovyx-processors/pom.xml                |  14 +-
 nifi-nar-bundles/nifi-groovyx-bundle/pom.xml       |   4 +-
 .../nifi-grpc-bundle/nifi-grpc-nar/pom.xml         |   8 +-
 .../nifi-grpc-bundle/nifi-grpc-processors/pom.xml  |   6 +-
 nifi-nar-bundles/nifi-grpc-bundle/pom.xml          |   6 +-
 .../nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml     |   4 +-
 .../nifi-hdfs-processors/pom.xml                   |  10 +-
 nifi-nar-bundles/nifi-hadoop-bundle/pom.xml        |   4 +-
 .../nifi-hadoop-libraries-nar/pom.xml              |   4 +-
 .../nifi-hadoop-libraries-bundle/pom.xml           |   2 +-
 .../nifi-hbase-bundle/nifi-hbase-nar/pom.xml       |   4 +-
 .../nifi-hbase-processors/pom.xml                  |  14 +-
 nifi-nar-bundles/nifi-hbase-bundle/pom.xml         |   4 +-
 .../nifi-hive-bundle/nifi-hive-nar/pom.xml         |   8 +-
 .../nifi-hive-bundle/nifi-hive-processors/pom.xml  |  14 +-
 .../nifi-hive-services-api-nar/pom.xml             |   8 +-
 .../nifi-hive-services-api/pom.xml                 |   6 +-
 .../nifi-hive-bundle/nifi-hive3-nar/pom.xml        |   8 +-
 .../nifi-hive-bundle/nifi-hive3-processors/pom.xml |  22 +-
 .../nifi-hive-bundle/nifi-hive_1_1-nar/pom.xml     |   8 +-
 .../nifi-hive_1_1-processors/pom.xml               |  14 +-
 nifi-nar-bundles/nifi-hive-bundle/pom.xml          |   4 +-
 .../nifi-hl7-bundle/nifi-hl7-nar/pom.xml           |   4 +-
 .../nifi-hl7-bundle/nifi-hl7-processors/pom.xml    |   6 +-
 nifi-nar-bundles/nifi-hl7-bundle/pom.xml           |   6 +-
 .../nifi-html-bundle/nifi-html-nar/pom.xml         |   4 +-
 .../nifi-html-bundle/nifi-html-processors/pom.xml  |   6 +-
 nifi-nar-bundles/nifi-html-bundle/pom.xml          |   4 +-
 .../nifi-ignite-bundle/nifi-ignite-nar/pom.xml     |   4 +-
 .../nifi-ignite-processors/pom.xml                 |   6 +-
 nifi-nar-bundles/nifi-ignite-bundle/pom.xml        |   4 +-
 .../nifi-influxdb-bundle/nifi-influxdb-nar/pom.xml |   4 +-
 .../nifi-influxdb-processors/pom.xml               |   6 +-
 nifi-nar-bundles/nifi-influxdb-bundle/pom.xml      |   4 +-
 nifi-nar-bundles/nifi-jetty-bundle/pom.xml         |   2 +-
 .../nifi-jms-cf-service-nar/pom.xml                |   6 +-
 .../nifi-jms-bundle/nifi-jms-cf-service/pom.xml    |   2 +-
 .../nifi-jms-processors-nar/pom.xml                |   6 +-
 .../nifi-jms-bundle/nifi-jms-processors/pom.xml    |   8 +-
 nifi-nar-bundles/nifi-jms-bundle/pom.xml           |   4 +-
 .../nifi-jolt-record-nar/pom.xml                   |   8 +-
 .../nifi-jolt-record-processors/pom.xml            |  14 +-
 nifi-nar-bundles/nifi-jolt-record-bundle/pom.xml   |  10 +-
 .../nifi-kafka-bundle/nifi-kafka-0-10-nar/pom.xml  |   4 +-
 .../nifi-kafka-0-10-processors/pom.xml             |   8 +-
 .../nifi-kafka-bundle/nifi-kafka-0-11-nar/pom.xml  |   4 +-
 .../nifi-kafka-0-11-processors/pom.xml             |   8 +-
 .../nifi-kafka-bundle/nifi-kafka-0-8-nar/pom.xml   |   2 +-
 .../nifi-kafka-0-8-processors/pom.xml              |   8 +-
 .../nifi-kafka-bundle/nifi-kafka-0-9-nar/pom.xml   |   4 +-
 .../nifi-kafka-0-9-processors/pom.xml              |   8 +-
 .../nifi-kafka-bundle/nifi-kafka-1-0-nar/pom.xml   |   4 +-
 .../nifi-kafka-1-0-processors/pom.xml              |  14 +-
 .../nifi-kafka-bundle/nifi-kafka-2-0-nar/pom.xml   |   4 +-
 .../nifi-kafka-2-0-processors/pom.xml              |  14 +-
 .../nifi-kafka-bundle/nifi-kafka-2-6-nar/pom.xml   |   4 +-
 .../nifi-kafka-2-6-processors/pom.xml              |  14 +-
 nifi-nar-bundles/nifi-kafka-bundle/pom.xml         |  16 +-
 .../nifi-kerberos-iaa-providers-nar/pom.xml        |   2 +-
 .../nifi-kerberos-iaa-providers/pom.xml            |   6 +-
 .../nifi-kerberos-iaa-providers-bundle/pom.xml     |   4 +-
 .../nifi-kite-bundle/nifi-kite-nar/pom.xml         |   4 +-
 .../nifi-kite-bundle/nifi-kite-processors/pom.xml  |   8 +-
 nifi-nar-bundles/nifi-kite-bundle/pom.xml          |   4 +-
 .../nifi-kudu-controller-service/pom.xml           |   6 +-
 .../nifi-kudu-bundle/nifi-kudu-nar/pom.xml         |  10 +-
 .../nifi-kudu-bundle/nifi-kudu-processors/pom.xml  |  14 +-
 nifi-nar-bundles/nifi-kudu-bundle/pom.xml          |   4 +-
 .../nifi-language-translation-nar/pom.xml          |   4 +-
 .../nifi-yandex-processors/pom.xml                 |   8 +-
 .../nifi-language-translation-bundle/pom.xml       |   2 +-
 .../nifi-ldap-iaa-providers-nar/pom.xml            |   2 +-
 .../nifi-ldap-iaa-providers/pom.xml                |   8 +-
 .../nifi-ldap-iaa-providers-bundle/pom.xml         |   4 +-
 .../nifi-lumberjack-nar/pom.xml                    |   8 +-
 .../nifi-lumberjack-processors/pom.xml             |  12 +-
 nifi-nar-bundles/nifi-lumberjack-bundle/pom.xml    |   4 +-
 .../nifi-media-bundle/nifi-image-viewer/pom.xml    |   2 +-
 .../nifi-media-bundle/nifi-media-nar/pom.xml       |   8 +-
 .../nifi-media-processors/pom.xml                  |   6 +-
 nifi-nar-bundles/nifi-media-bundle/pom.xml         |   4 +-
 .../nifi-metrics-reporter-service-api-nar/pom.xml  |   6 +-
 .../nifi-metrics-reporter-service-api/pom.xml      |   2 +-
 .../nifi-metrics-reporting-nar/pom.xml             |   6 +-
 .../nifi-metrics-reporting-task/pom.xml            |   8 +-
 .../nifi-metrics-reporting-bundle/pom.xml          |   6 +-
 .../nifi-mongodb-client-service-api-nar/pom.xml    |   6 +-
 .../nifi-mongodb-client-service-api/pom.xml        |   6 +-
 .../nifi-mongodb-bundle/nifi-mongodb-nar/pom.xml   |   4 +-
 .../nifi-mongodb-processors/pom.xml                |  18 +-
 .../nifi-mongodb-services-nar/pom.xml              |   6 +-
 .../nifi-mongodb-services/pom.xml                  |  12 +-
 nifi-nar-bundles/nifi-mongodb-bundle/pom.xml       |   4 +-
 .../nifi-mqtt-bundle/nifi-mqtt-nar/pom.xml         |   4 +-
 .../nifi-mqtt-bundle/nifi-mqtt-processors/pom.xml  |   6 +-
 nifi-nar-bundles/nifi-mqtt-bundle/pom.xml          |   4 +-
 .../nifi-network-processors-nar/pom.xml            |   4 +-
 .../nifi-network-processors/pom.xml                |   8 +-
 .../nifi-network-bundle/nifi-network-utils/pom.xml |   2 +-
 nifi-nar-bundles/nifi-network-bundle/pom.xml       |   2 +-
 .../nifi-parquet-bundle/nifi-parquet-nar/pom.xml   |   8 +-
 .../nifi-parquet-processors/pom.xml                |  10 +-
 nifi-nar-bundles/nifi-parquet-bundle/pom.xml       |   4 +-
 .../nifi-poi-bundle/nifi-poi-nar/pom.xml           |   6 +-
 .../nifi-poi-bundle/nifi-poi-processors/pom.xml    |   8 +-
 nifi-nar-bundles/nifi-poi-bundle/pom.xml           |   4 +-
 .../nifi-prometheus-nar/pom.xml                    |   6 +-
 .../nifi-prometheus-reporting-task/pom.xml         |  16 +-
 nifi-nar-bundles/nifi-prometheus-bundle/pom.xml    |   4 +-
 .../nifi-persistent-provenance-repository/pom.xml  |  10 +-
 .../nifi-provenance-repository-nar/pom.xml         |   2 +-
 .../nifi-volatile-provenance-repository/pom.xml    |   6 +-
 .../nifi-provenance-repository-bundle/pom.xml      |   6 +-
 .../nifi-ranger-bundle/nifi-ranger-nar/pom.xml     |   6 +-
 .../nifi-ranger-bundle/nifi-ranger-plugin/pom.xml  |   6 +-
 .../nifi-ranger-resources/pom.xml                  |   2 +-
 nifi-nar-bundles/nifi-ranger-bundle/pom.xml        |   4 +-
 .../nifi-redis-extensions/pom.xml                  |   8 +-
 .../nifi-redis-bundle/nifi-redis-nar/pom.xml       |   8 +-
 .../nifi-redis-service-api-nar/pom.xml             |   8 +-
 .../nifi-redis-service-api/pom.xml                 |   2 +-
 nifi-nar-bundles/nifi-redis-bundle/pom.xml         |   4 +-
 .../nifi-registry-bundle/nifi-registry-nar/pom.xml |   6 +-
 .../nifi-registry-service/pom.xml                  |   6 +-
 nifi-nar-bundles/nifi-registry-bundle/pom.xml      |   2 +-
 .../nifi-rethinkdb-nar/pom.xml                     |   4 +-
 .../nifi-rethinkdb-processors/pom.xml              |   8 +-
 nifi-nar-bundles/nifi-rethinkdb-bundle/pom.xml     |   4 +-
 .../nifi-riemann-bundle/nifi-riemann-nar/pom.xml   |   2 +-
 .../nifi-riemann-processors/pom.xml                |   6 +-
 nifi-nar-bundles/nifi-riemann-bundle/pom.xml       |   4 +-
 .../nifi-rules-action-handler-nar/pom.xml          |   8 +-
 .../nifi-rules-action-handler-service/pom.xml      |  12 +-
 .../nifi-rules-action-handler-bundle/pom.xml       |   4 +-
 .../nifi-scripting-nar/pom.xml                     |   4 +-
 .../nifi-scripting-processors/pom.xml              |  12 +-
 nifi-nar-bundles/nifi-scripting-bundle/pom.xml     |  14 +-
 .../nifi-site-to-site-reporting-nar/pom.xml        |   4 +-
 .../nifi-site-to-site-reporting-task/pom.xml       |  22 +-
 .../nifi-site-to-site-reporting-bundle/pom.xml     |   4 +-
 .../nifi-slack-bundle/nifi-slack-nar/pom.xml       |   6 +-
 .../nifi-slack-processors/pom.xml                  |   8 +-
 nifi-nar-bundles/nifi-slack-bundle/pom.xml         |   4 +-
 .../nifi-smb-bundle/nifi-smb-nar/pom.xml           |   6 +-
 .../nifi-smb-bundle/nifi-smb-processors/pom.xml    |   6 +-
 nifi-nar-bundles/nifi-smb-bundle/pom.xml           |   4 +-
 .../nifi-snmp-bundle/nifi-snmp-nar/pom.xml         |   2 +-
 .../nifi-snmp-bundle/nifi-snmp-processors/pom.xml  |   6 +-
 nifi-nar-bundles/nifi-snmp-bundle/pom.xml          |   4 +-
 .../nifi-social-media-nar/pom.xml                  |   4 +-
 .../nifi-twitter-processors/pom.xml                |   6 +-
 nifi-nar-bundles/nifi-social-media-bundle/pom.xml  |   2 +-
 .../nifi-solr-bundle/nifi-solr-nar/pom.xml         |   6 +-
 .../nifi-solr-bundle/nifi-solr-processors/pom.xml  |  12 +-
 nifi-nar-bundles/nifi-solr-bundle/pom.xml          |   2 +-
 .../nifi-livy-controller-service-api-nar/pom.xml   |   6 +-
 .../nifi-livy-controller-service-api/pom.xml       |   2 +-
 .../nifi-livy-controller-service/pom.xml           |   6 +-
 .../nifi-spark-bundle/nifi-livy-nar/pom.xml        |   8 +-
 .../nifi-spark-bundle/nifi-livy-processors/pom.xml |  12 +-
 nifi-nar-bundles/nifi-spark-bundle/pom.xml         |   6 +-
 .../nifi-splunk-bundle/nifi-splunk-nar/pom.xml     |   8 +-
 .../nifi-splunk-processors/pom.xml                 |   6 +-
 nifi-nar-bundles/nifi-splunk-bundle/pom.xml        |   4 +-
 .../nifi-spring-bundle/nifi-spring-nar/pom.xml     |   2 +-
 .../nifi-spring-processors/pom.xml                 |   6 +-
 nifi-nar-bundles/nifi-spring-bundle/pom.xml        |   4 +-
 .../nifi-sql-reporting-nar/pom.xml                 |   8 +-
 .../nifi-sql-reporting-tasks/pom.xml               |  18 +-
 nifi-nar-bundles/nifi-sql-reporting-bundle/pom.xml |   4 +-
 .../nifi-jolt-transform-json-ui/pom.xml            |  16 +-
 .../nifi-standard-content-viewer/pom.xml           |   4 +-
 .../nifi-standard-bundle/nifi-standard-nar/pom.xml |   4 +-
 .../nifi-standard-processors/pom.xml               |  44 ++--
 .../nifi-standard-reporting-tasks/pom.xml          |   8 +-
 .../nifi-standard-utils/pom.xml                    |   4 +-
 .../nifi-standard-web-test-utils/pom.xml           |   4 +-
 nifi-nar-bundles/nifi-standard-bundle/pom.xml      |  12 +-
 .../nifi-dbcp-service-api/pom.xml                  |   4 +-
 .../nifi-dbcp-service-nar/pom.xml                  |   6 +-
 .../nifi-dbcp-service/pom.xml                      |  22 +-
 .../nifi-dbcp-service-bundle/pom.xml               |   2 +-
 .../pom.xml                                        |   2 +-
 .../nifi-distributed-cache-client-service/pom.xml  |   8 +-
 .../nifi-distributed-cache-protocol/pom.xml        |   4 +-
 .../nifi-distributed-cache-server/pom.xml          |  12 +-
 .../nifi-distributed-cache-services-nar/pom.xml    |   4 +-
 .../nifi-distributed-cache-services-bundle/pom.xml |   2 +-
 .../nifi-hadoop-dbcp-service-nar/pom.xml           |   6 +-
 .../nifi-hadoop-dbcp-service/pom.xml               |  18 +-
 .../nifi-hadoop-dbcp-service-bundle/pom.xml        |   2 +-
 .../nifi-hbase-client-service-api/pom.xml          |   2 +-
 .../nifi-hbase_1_1_2-client-service-nar/pom.xml    |   8 +-
 .../nifi-hbase_1_1_2-client-service/pom.xml        |  10 +-
 .../nifi-hbase_1_1_2-client-service-bundle/pom.xml |   4 +-
 .../nifi-hbase_2-client-service-nar/pom.xml        |   8 +-
 .../nifi-hbase_2-client-service/pom.xml            |  10 +-
 .../nifi-hbase_2-client-service-bundle/pom.xml     |   4 +-
 .../nifi-http-context-map-api/pom.xml              |   2 +-
 .../nifi-http-context-map-nar/pom.xml              |   4 +-
 .../nifi-http-context-map/pom.xml                  |   4 +-
 .../nifi-http-context-map-bundle/pom.xml           |   2 +-
 .../nifi-hwx-schema-registry-nar/pom.xml           |   6 +-
 .../nifi-hwx-schema-registry-service/pom.xml       |   8 +-
 .../nifi-hwx-schema-registry-bundle/pom.xml        |   2 +-
 .../nifi-kerberos-credentials-service-api/pom.xml  |   2 +-
 .../nifi-kerberos-credentials-service-nar/pom.xml  |   6 +-
 .../nifi-kerberos-credentials-service/pom.xml      |   4 +-
 .../pom.xml                                        |   2 +-
 .../nifi-load-distribution-service-api/pom.xml     |   2 +-
 .../nifi-lookup-service-api/pom.xml                |   2 +-
 .../nifi-lookup-services-nar/pom.xml               |   6 +-
 .../nifi-lookup-services/pom.xml                   |  24 +-
 .../nifi-lookup-services-bundle/pom.xml            |   2 +-
 .../nifi-oauth2-provider-api/pom.xml               |   2 +-
 .../nifi-oauth2-provider-nar/pom.xml               |   8 +-
 .../nifi-oauth2-provider-service/pom.xml           |  14 +-
 .../nifi-oauth2-provider-bundle/pom.xml            |   4 +-
 .../nifi-proxy-configuration-api/pom.xml           |   4 +-
 .../nifi-proxy-configuration-nar/pom.xml           |   6 +-
 .../nifi-proxy-configuration/pom.xml               |   4 +-
 .../nifi-proxy-configuration-bundle/pom.xml        |   2 +-
 .../nifi-record-serialization-service-api/pom.xml  |   2 +-
 .../nifi-record-serialization-services-nar/pom.xml |   6 +-
 .../nifi-record-serialization-services/pom.xml     |  10 +-
 .../pom.xml                                        |   2 +-
 .../nifi-record-sink-api/pom.xml                   |   2 +-
 .../nifi-record-sink-service-nar/pom.xml           |   6 +-
 .../nifi-record-sink-service/pom.xml               |   6 +-
 .../nifi-record-sink-service-bundle/pom.xml        |   2 +-
 .../nifi-rules-engine-service-api/pom.xml          |   4 +-
 .../nifi-schema-registry-service-api/pom.xml       |   2 +-
 .../nifi-ssl-context-nar/pom.xml                   |   4 +-
 .../nifi-ssl-context-service/pom.xml               |   6 +-
 .../nifi-ssl-context-bundle/pom.xml                |   2 +-
 .../nifi-ssl-context-service-api/pom.xml           |   4 +-
 .../nifi-standard-services-api-nar/pom.xml         |  14 +-
 nifi-nar-bundles/nifi-standard-services/pom.xml    |   2 +-
 .../nifi-stateful-analysis-nar/pom.xml             |   4 +-
 .../nifi-stateful-analysis-processors/pom.xml      |   8 +-
 .../nifi-stateful-analysis-bundle/pom.xml          |   4 +-
 .../nifi-tcp-bundle/nifi-tcp-nar/pom.xml           |   6 +-
 .../nifi-tcp-bundle/nifi-tcp-processors/pom.xml    |   6 +-
 nifi-nar-bundles/nifi-tcp-bundle/pom.xml           |   2 +-
 .../nifi-update-attribute-model/pom.xml            |   4 +-
 .../nifi-update-attribute-nar/pom.xml              |   4 +-
 .../nifi-update-attribute-processor/pom.xml        |   6 +-
 .../nifi-update-attribute-ui/pom.xml               |  12 +-
 .../nifi-update-attribute-bundle/pom.xml           |   8 +-
 .../nifi-websocket-processors-nar/pom.xml          |   8 +-
 .../nifi-websocket-processors/pom.xml              |   8 +-
 .../nifi-websocket-services-api-nar/pom.xml        |   8 +-
 .../nifi-websocket-services-api/pom.xml            |   6 +-
 .../nifi-websocket-services-jetty-nar/pom.xml      |   8 +-
 .../nifi-websocket-services-jetty/pom.xml          |   6 +-
 nifi-nar-bundles/nifi-websocket-bundle/pom.xml     |   4 +-
 .../nifi-windows-event-log-nar/pom.xml             |   4 +-
 .../nifi-windows-event-log-processors/pom.xml      |   6 +-
 .../nifi-windows-event-log-bundle/pom.xml          |   4 +-
 nifi-nar-bundles/pom.xml                           |  54 ++---
 .../nifi-system-test-extensions-nar/pom.xml        |   8 +-
 .../nifi-system-test-extensions/pom.xml            |   6 +-
 .../nifi-system-test-extensions-bundle/pom.xml     |   2 +-
 .../nifi-system-test-extensions2-nar/pom.xml       |   8 +-
 .../nifi-system-test-extensions2/pom.xml           |   6 +-
 .../nifi-system-test-extensions2-bundle/pom.xml    |   2 +-
 nifi-system-tests/nifi-system-test-suite/pom.xml   |  26 +--
 nifi-system-tests/pom.xml                          |   2 +-
 nifi-toolkit/nifi-toolkit-admin/pom.xml            |  14 +-
 nifi-toolkit/nifi-toolkit-api/pom.xml              |   2 +-
 nifi-toolkit/nifi-toolkit-assembly/pom.xml         |  16 +-
 nifi-toolkit/nifi-toolkit-cli/pom.xml              |   4 +-
 nifi-toolkit/nifi-toolkit-encrypt-config/pom.xml   |  10 +-
 nifi-toolkit/nifi-toolkit-flowanalyzer/pom.xml     |   4 +-
 nifi-toolkit/nifi-toolkit-flowfile-repo/pom.xml    |   4 +-
 nifi-toolkit/nifi-toolkit-s2s/pom.xml              |   4 +-
 nifi-toolkit/nifi-toolkit-tls/pom.xml              |   8 +-
 .../nifi-toolkit-zookeeper-migrator/pom.xml        |   2 +-
 nifi-toolkit/pom.xml                               |   2 +-
 pom.xml                                            |   2 +-
 487 files changed, 1686 insertions(+), 1686 deletions(-)

diff --git a/nifi-api/pom.xml b/nifi-api/pom.xml
index 975797a..3973c80 100644
--- a/nifi-api/pom.xml
+++ b/nifi-api/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-api</artifactId>
     <packaging>jar</packaging>
diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml
index 119d7c3..bc50d5a 100644
--- a/nifi-assembly/pom.xml
+++ b/nifi-assembly/pom.xml
@@ -14,7 +14,7 @@ language governing permissions and limitations under the License. -->
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-assembly</artifactId>
     <packaging>pom</packaging>
@@ -94,28 +94,28 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-framework-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-runtime</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-bootstrap</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-resources</artifactId>
             <classifier>resources</classifier>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>runtime</scope>
             <type>zip</type>
         </dependency>
@@ -123,614 +123,614 @@ language governing permissions and limitations under the License. -->
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-docs</artifactId>
             <classifier>resources</classifier>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>runtime</scope>
             <type>zip</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-framework-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-provenance-repository-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-ssl-context-service-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-distributed-cache-services-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-datadog-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-jetty-bundle</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-update-attribute-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hadoop-libraries-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hadoop-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-kafka-0-11-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-kafka-1-0-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-kafka-2-0-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-kafka-2-6-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-confluent-platform-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-http-context-map-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-html-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-lookup-services-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-poi-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-kudu-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-ldap-iaa-providers-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-kerberos-iaa-providers-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-dbcp-service-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hadoop-dbcp-service-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
 	<dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mongodb-client-service-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mongodb-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mongodb-services-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-solr-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-social-media-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hl7-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-ccda-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-language-translation-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-enrich-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-aws-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-aws-service-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-ambari-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-ignite-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-rethinkdb-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-prometheus-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-influxdb-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-network-processors-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-avro-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-couchbase-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-couchbase-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hbase-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-riemann-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hbase_1_1_2-client-service-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hbase_2-client-service-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-oauth2-provider-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-azure-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-azure-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-scripting-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-groovyx-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-elasticsearch-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-elasticsearch-client-service-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-elasticsearch-client-service-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-elasticsearch-restapi-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-lumberjack-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-beats-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-cybersecurity-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-email-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-amqp-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-splunk-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-jms-cf-service-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-jms-processors-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-cassandra-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-cassandra-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-cassandra-services-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-spring-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-registry-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hive-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hive-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-site-to-site-reporting-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-serialization-services-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mqtt-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-snmp-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-evtx-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-slack-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-smb-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-windows-event-log-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-websocket-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-websocket-services-jetty-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-websocket-processors-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-tcp-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-gcp-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-gcp-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-stateful-analysis-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-cdc-mysql-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-parquet-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hwx-schema-registry-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-redis-service-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-redis-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-metrics-reporter-service-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-metrics-reporting-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-livy-controller-service-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-livy-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-kerberos-credentials-service-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-proxy-configuration-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-jolt-record-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-sink-service-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <!-- dependencies for jaxb/activation/annotation for running NiFi on Java 11 -->
@@ -771,7 +771,7 @@ language governing permissions and limitations under the License. -->
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-kafka-0-9-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
             </dependencies>
@@ -785,7 +785,7 @@ language governing permissions and limitations under the License. -->
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-kafka-0-10-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
             </dependencies>
@@ -799,7 +799,7 @@ language governing permissions and limitations under the License. -->
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-elasticsearch-5-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
             </dependencies>
@@ -813,25 +813,25 @@ language governing permissions and limitations under the License. -->
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-graph-client-service-api-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-neo4j-cypher-service-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-graph-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-other-graph-services-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
             </dependencies>
@@ -848,7 +848,7 @@ language governing permissions and limitations under the License. -->
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-grpc-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
             </dependencies>
@@ -865,7 +865,7 @@ language governing permissions and limitations under the License. -->
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-atlas-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
             </dependencies>
@@ -882,7 +882,7 @@ language governing permissions and limitations under the License. -->
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-hive_1_1-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
             </dependencies>
@@ -899,7 +899,7 @@ language governing permissions and limitations under the License. -->
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-hive3-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
             </dependencies>
@@ -914,13 +914,13 @@ language governing permissions and limitations under the License. -->
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-easyrules-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-rules-action-handler-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
             </dependencies>
@@ -935,19 +935,19 @@ language governing permissions and limitations under the License. -->
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-sql-reporting-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-easyrules-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-rules-action-handler-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
             </dependencies>
@@ -962,19 +962,19 @@ language governing permissions and limitations under the License. -->
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-accumulo-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
 	    	</dependency>
 		<dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-accumulo-services-api-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
 		</dependency>
 		<dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-accumulo-services-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
             </dependencies>
@@ -1294,7 +1294,7 @@ language governing permissions and limitations under the License. -->
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-ranger-nar</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                     <type>nar</type>
                 </dependency>
                 <dependency>
@@ -1305,7 +1305,7 @@ language governing permissions and limitations under the License. -->
                 <dependency>
                     <groupId>org.apache.nifi</groupId>
                     <artifactId>nifi-ranger-resources</artifactId>
-                    <version>1.12.0</version>
+                    <version>1.12.1-SNAPSHOT</version>
                 </dependency>
             </dependencies>
         </profile>
diff --git a/nifi-bootstrap/pom.xml b/nifi-bootstrap/pom.xml
index 4937c82..0da05f9 100644
--- a/nifi-bootstrap/pom.xml
+++ b/nifi-bootstrap/pom.xml
@@ -14,7 +14,7 @@ language governing permissions and limitations under the License. -->
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-bootstrap</artifactId>
     <packaging>jar</packaging>
@@ -27,18 +27,18 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>javax.mail</groupId>
@@ -48,7 +48,7 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-expression-language</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>com.squareup.okhttp3</groupId>
@@ -66,7 +66,7 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-properties-loader</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>net.java.dev.jna</groupId>
diff --git a/nifi-commons/nifi-data-provenance-utils/pom.xml b/nifi-commons/nifi-data-provenance-utils/pom.xml
index d17feee..7199f8b 100644
--- a/nifi-commons/nifi-data-provenance-utils/pom.xml
+++ b/nifi-commons/nifi-data-provenance-utils/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-commons</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-data-provenance-utils</artifactId>
     <packaging>jar</packaging>
@@ -25,22 +25,22 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-framework-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.bouncycastle</groupId>
@@ -50,12 +50,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-properties-loader</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-properties</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
diff --git a/nifi-commons/nifi-expression-language/pom.xml b/nifi-commons/nifi-expression-language/pom.xml
index 0f5f0b9..1738d7a 100644
--- a/nifi-commons/nifi-expression-language/pom.xml
+++ b/nifi-commons/nifi-expression-language/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-commons</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-expression-language</artifactId>
     <packaging>jar</packaging>
@@ -61,17 +61,17 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-parameter</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.hamcrest</groupId>
diff --git a/nifi-commons/nifi-flowfile-packager/pom.xml b/nifi-commons/nifi-flowfile-packager/pom.xml
index 8fc1c44..6d03c0a 100644
--- a/nifi-commons/nifi-flowfile-packager/pom.xml
+++ b/nifi-commons/nifi-flowfile-packager/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-commons</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-flowfile-packager</artifactId>
     <packaging>jar</packaging>
diff --git a/nifi-commons/nifi-hl7-query-language/pom.xml b/nifi-commons/nifi-hl7-query-language/pom.xml
index bcd602d..5388063 100644
--- a/nifi-commons/nifi-hl7-query-language/pom.xml
+++ b/nifi-commons/nifi-hl7-query-language/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-commons</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-hl7-query-language</artifactId>
diff --git a/nifi-commons/nifi-json-utils/pom.xml b/nifi-commons/nifi-json-utils/pom.xml
index 46dd4e3..be501da 100644
--- a/nifi-commons/nifi-json-utils/pom.xml
+++ b/nifi-commons/nifi-json-utils/pom.xml
@@ -18,16 +18,16 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-commons</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-json-utils</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>jar</packaging>
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>com.fasterxml.jackson.core</groupId>
diff --git a/nifi-commons/nifi-logging-utils/pom.xml b/nifi-commons/nifi-logging-utils/pom.xml
index b32c573..7f6e75c 100644
--- a/nifi-commons/nifi-logging-utils/pom.xml
+++ b/nifi-commons/nifi-logging-utils/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-commons</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-logging-utils</artifactId>
     <description>Utilities for logging</description>
diff --git a/nifi-commons/nifi-metrics/pom.xml b/nifi-commons/nifi-metrics/pom.xml
index 5915645..9c1d3f7 100644
--- a/nifi-commons/nifi-metrics/pom.xml
+++ b/nifi-commons/nifi-metrics/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <artifactId>nifi-commons</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
@@ -63,7 +63,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>io.dropwizard.metrics</groupId>
diff --git a/nifi-commons/nifi-parameter/pom.xml b/nifi-commons/nifi-parameter/pom.xml
index 15cd682..bb651ff 100644
--- a/nifi-commons/nifi-parameter/pom.xml
+++ b/nifi-commons/nifi-parameter/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <artifactId>nifi-commons</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
@@ -28,7 +28,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-commons/nifi-properties/pom.xml b/nifi-commons/nifi-properties/pom.xml
index 841a1e1..3bc373b 100644
--- a/nifi-commons/nifi-properties/pom.xml
+++ b/nifi-commons/nifi-properties/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-commons</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-properties</artifactId>
 </project>
diff --git a/nifi-commons/nifi-record-path/pom.xml b/nifi-commons/nifi-record-path/pom.xml
index be9febd..0e92347 100644
--- a/nifi-commons/nifi-record-path/pom.xml
+++ b/nifi-commons/nifi-record-path/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-commons</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
   </parent>
 
   <artifactId>nifi-record-path</artifactId>
@@ -59,17 +59,17 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-properties</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.antlr</groupId>
diff --git a/nifi-commons/nifi-record/pom.xml b/nifi-commons/nifi-record/pom.xml
index 4d7ce98..72cee43 100644
--- a/nifi-commons/nifi-record/pom.xml
+++ b/nifi-commons/nifi-record/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-commons</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-record</artifactId>
diff --git a/nifi-commons/nifi-rocksdb-utils/pom.xml b/nifi-commons/nifi-rocksdb-utils/pom.xml
index ab184cf..f04fca0 100644
--- a/nifi-commons/nifi-rocksdb-utils/pom.xml
+++ b/nifi-commons/nifi-rocksdb-utils/pom.xml
@@ -18,21 +18,21 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-commons</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-rocksdb-utils</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>jar</packaging>
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-properties</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.rocksdb</groupId>
diff --git a/nifi-commons/nifi-schema-utils/pom.xml b/nifi-commons/nifi-schema-utils/pom.xml
index f59b200..73608d3 100644
--- a/nifi-commons/nifi-schema-utils/pom.xml
+++ b/nifi-commons/nifi-schema-utils/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-commons</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-schema-utils</artifactId>
     <dependencies>
diff --git a/nifi-commons/nifi-security-utils/pom.xml b/nifi-commons/nifi-security-utils/pom.xml
index ca80348..cf1b5d3 100644
--- a/nifi-commons/nifi-security-utils/pom.xml
+++ b/nifi-commons/nifi-security-utils/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-commons</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-security-utils</artifactId>
     <description>
@@ -30,12 +30,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>ch.qos.logback</groupId>
@@ -65,7 +65,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-properties</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>at.favre.lib</groupId>
diff --git a/nifi-commons/nifi-site-to-site-client/pom.xml b/nifi-commons/nifi-site-to-site-client/pom.xml
index e6236f9..c28dc49 100644
--- a/nifi-commons/nifi-site-to-site-client/pom.xml
+++ b/nifi-commons/nifi-site-to-site-client/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-commons</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-site-to-site-client</artifactId>
@@ -34,22 +34,22 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-framework-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.commons</groupId>
@@ -64,7 +64,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-client-dto</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>${httpclient.group}</groupId>
diff --git a/nifi-commons/nifi-socket-utils/pom.xml b/nifi-commons/nifi-socket-utils/pom.xml
index 3c2013e..d5280bc 100644
--- a/nifi-commons/nifi-socket-utils/pom.xml
+++ b/nifi-commons/nifi-socket-utils/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-commons</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-socket-utils</artifactId>
     <description>Utilities for socket communication</description>
@@ -30,12 +30,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-logging-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>commons-net</groupId>
@@ -55,7 +55,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-properties</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
diff --git a/nifi-commons/nifi-utils/pom.xml b/nifi-commons/nifi-utils/pom.xml
index b498744..040f858 100644
--- a/nifi-commons/nifi-utils/pom.xml
+++ b/nifi-commons/nifi-utils/pom.xml
@@ -18,10 +18,10 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-commons</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-utils</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>jar</packaging>
     <description>
         This nifi-utils module should be a general purpose place to store widely
@@ -38,7 +38,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
     <profiles>
diff --git a/nifi-commons/nifi-web-utils/pom.xml b/nifi-commons/nifi-web-utils/pom.xml
index 8a44134..c6e48e1 100644
--- a/nifi-commons/nifi-web-utils/pom.xml
+++ b/nifi-commons/nifi-web-utils/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-commons</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-web-utils</artifactId>
     <properties>
@@ -29,7 +29,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>commons-codec</groupId>
diff --git a/nifi-commons/nifi-write-ahead-log/pom.xml b/nifi-commons/nifi-write-ahead-log/pom.xml
index cdf5596..4d72e43 100644
--- a/nifi-commons/nifi-write-ahead-log/pom.xml
+++ b/nifi-commons/nifi-write-ahead-log/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-commons</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-write-ahead-log</artifactId>
     <packaging>jar</packaging>
@@ -30,7 +30,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
diff --git a/nifi-commons/pom.xml b/nifi-commons/pom.xml
index 9ac821c..4d618ce 100644
--- a/nifi-commons/pom.xml
+++ b/nifi-commons/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-commons</artifactId>
diff --git a/nifi-docker/docker-compose/docker-compose.yml b/nifi-docker/docker-compose/docker-compose.yml
index 42e1006..a28afc3 100644
--- a/nifi-docker/docker-compose/docker-compose.yml
+++ b/nifi-docker/docker-compose/docker-compose.yml
@@ -22,7 +22,7 @@ services:
     environment:
       - ALLOW_ANONYMOUS_LOGIN=yes
   nifi:
-    image: apache/nifi:1.12.0-SNAPSHOT-dockermaven
+    image: apache/nifi:1.12.1-dockermaven
     ports:
       - 8080 # Unsecured HTTP Web Port
     environment:
diff --git a/nifi-docker/dockerhub/DockerImage.txt b/nifi-docker/dockerhub/DockerImage.txt
index dc7758d..5835f72 100644
--- a/nifi-docker/dockerhub/DockerImage.txt
+++ b/nifi-docker/dockerhub/DockerImage.txt
@@ -13,4 +13,4 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-apache/nifi:1.12.0
+apache/nifi:1.12.1
diff --git a/nifi-docker/dockerhub/Dockerfile b/nifi-docker/dockerhub/Dockerfile
index 41ebe42..64a2a4a 100644
--- a/nifi-docker/dockerhub/Dockerfile
+++ b/nifi-docker/dockerhub/Dockerfile
@@ -22,7 +22,7 @@ LABEL site="https://nifi.apache.org"
 
 ARG UID=1000
 ARG GID=1000
-ARG NIFI_VERSION=1.12.0
+ARG NIFI_VERSION=1.12.1
 ARG BASE_URL=https://archive.apache.org/dist
 ARG MIRROR_BASE_URL=${MIRROR_BASE_URL:-${BASE_URL}}
 ARG NIFI_BINARY_PATH=${NIFI_BINARY_PATH:-/nifi/${NIFI_VERSION}/nifi-${NIFI_VERSION}-bin.zip}
diff --git a/nifi-docker/dockerhub/README.md b/nifi-docker/dockerhub/README.md
index beacb75..5dfb688 100644
--- a/nifi-docker/dockerhub/README.md
+++ b/nifi-docker/dockerhub/README.md
@@ -19,7 +19,7 @@
 - The NiFi Toolkit has been added to the image under the path `/opt/nifi/nifi-toolkit-current` also set as the environment variable `NIFI_TOOLKIT_HOME`
 - The installation directory and related environment variables are changed to be version-agnostic to `/opt/nifi/nifi-current`:
 ```
-docker run --rm --entrypoint /bin/bash apache/nifi:1.12.0 -c 'env | grep NIFI'
+docker run --rm --entrypoint /bin/bash apache/nifi:1.12.1 -c 'env | grep NIFI'
 NIFI_HOME=/opt/nifi/nifi-current
 NIFI_LOG_DIR=/opt/nifi/nifi-current/logs
 NIFI_TOOLKIT_HOME=/opt/nifi/nifi-toolkit-current
@@ -28,7 +28,7 @@ NIFI_BASE_DIR=/opt/nifi
 ```
 - A symlink refer to the new path for backward compatibility:
 ```
-docker run --rm --entrypoint /bin/bash apache/nifi:1.12.0 -c 'readlink /opt/nifi/nifi-1.12.0'                                   /opt/nifi/nifi-current
+docker run --rm --entrypoint /bin/bash apache/nifi:1.12.1 -c 'readlink /opt/nifi/nifi-1.12.1'                                   /opt/nifi/nifi-current
 ```
 
 # Docker Image Quickstart
diff --git a/nifi-docker/dockerhub/pom.xml b/nifi-docker/dockerhub/pom.xml
index 485bdbc..714e732 100644
--- a/nifi-docker/dockerhub/pom.xml
+++ b/nifi-docker/dockerhub/pom.xml
@@ -15,7 +15,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-docker</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>dockerhub</artifactId>
diff --git a/nifi-docker/dockermaven-stateless/pom.xml b/nifi-docker/dockermaven-stateless/pom.xml
index f8f5f75..3e2ada4 100644
--- a/nifi-docker/dockermaven-stateless/pom.xml
+++ b/nifi-docker/dockermaven-stateless/pom.xml
@@ -15,7 +15,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-docker</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>dockermaven-stateless</artifactId>
diff --git a/nifi-docker/dockermaven/pom.xml b/nifi-docker/dockermaven/pom.xml
index 667a2dd..427350f 100644
--- a/nifi-docker/dockermaven/pom.xml
+++ b/nifi-docker/dockermaven/pom.xml
@@ -15,7 +15,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-docker</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>dockermaven</artifactId>
diff --git a/nifi-docker/pom.xml b/nifi-docker/pom.xml
index 514b5f0..6ecc68f 100644
--- a/nifi-docker/pom.xml
+++ b/nifi-docker/pom.xml
@@ -15,12 +15,12 @@ language governing permissions and limitations under the License. -->
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-docker</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>
diff --git a/nifi-docs/pom.xml b/nifi-docs/pom.xml
index fea5ee6..9aabd1c 100644
--- a/nifi-docs/pom.xml
+++ b/nifi-docs/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <packaging>pom</packaging>
     <artifactId>nifi-docs</artifactId>
diff --git a/nifi-external/nifi-example-bundle/nifi-nifi-example-nar/pom.xml b/nifi-external/nifi-example-bundle/nifi-nifi-example-nar/pom.xml
index 9de472f..9c87a78 100644
--- a/nifi-external/nifi-example-bundle/nifi-nifi-example-nar/pom.xml
+++ b/nifi-external/nifi-example-bundle/nifi-nifi-example-nar/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-example-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-example-nar</artifactId>
diff --git a/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/pom.xml b/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/pom.xml
index 2bdcb7d..8deee42 100644
--- a/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/pom.xml
+++ b/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-example-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-nifi-example-processors</artifactId>
@@ -29,17 +29,17 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-external/nifi-example-bundle/pom.xml b/nifi-external/nifi-example-bundle/pom.xml
index fcd56d4..1585cea 100644
--- a/nifi-external/nifi-example-bundle/pom.xml
+++ b/nifi-external/nifi-example-bundle/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-external</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-example-bundle</artifactId>
@@ -35,7 +35,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-nifi-example-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-external/nifi-spark-receiver/pom.xml b/nifi-external/nifi-spark-receiver/pom.xml
index 9ff276b..6a9debf 100644
--- a/nifi-external/nifi-spark-receiver/pom.xml
+++ b/nifi-external/nifi-spark-receiver/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-external</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-spark-receiver</artifactId>
@@ -33,7 +33,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-site-to-site-client</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>com.sun.jersey</groupId>
diff --git a/nifi-external/nifi-storm-spout/pom.xml b/nifi-external/nifi-storm-spout/pom.xml
index 0ae61ff..615355c 100644
--- a/nifi-external/nifi-storm-spout/pom.xml
+++ b/nifi-external/nifi-storm-spout/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-external</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-storm-spout</artifactId>
 
@@ -37,7 +37,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-site-to-site-client</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
diff --git a/nifi-external/pom.xml b/nifi-external/pom.xml
index 1a154a1..a308456 100644
--- a/nifi-external/pom.xml
+++ b/nifi-external/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-external</artifactId>
diff --git a/nifi-framework-api/pom.xml b/nifi-framework-api/pom.xml
index 1e080c4..864eb35 100644
--- a/nifi-framework-api/pom.xml
+++ b/nifi-framework-api/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-framework-api</artifactId>
     <packaging>jar</packaging>
@@ -27,12 +27,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-maven-archetypes/nifi-processor-bundle-archetype/pom.xml b/nifi-maven-archetypes/nifi-processor-bundle-archetype/pom.xml
index f471ea0..9cce4ba 100644
--- a/nifi-maven-archetypes/nifi-processor-bundle-archetype/pom.xml
+++ b/nifi-maven-archetypes/nifi-processor-bundle-archetype/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-maven-archetypes</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-processor-bundle-archetype</artifactId>
diff --git a/nifi-maven-archetypes/nifi-service-bundle-archetype/pom.xml b/nifi-maven-archetypes/nifi-service-bundle-archetype/pom.xml
index f57f262..9de4440 100644
--- a/nifi-maven-archetypes/nifi-service-bundle-archetype/pom.xml
+++ b/nifi-maven-archetypes/nifi-service-bundle-archetype/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-maven-archetypes</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-service-bundle-archetype</artifactId>
diff --git a/nifi-maven-archetypes/pom.xml b/nifi-maven-archetypes/pom.xml
index 919be3f..4b357d4 100644
--- a/nifi-maven-archetypes/pom.xml
+++ b/nifi-maven-archetypes/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-maven-archetypes</artifactId>
diff --git a/nifi-mock/pom.xml b/nifi-mock/pom.xml
index e452153..a39faa6 100644
--- a/nifi-mock/pom.xml
+++ b/nifi-mock/pom.xml
@@ -18,34 +18,34 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-mock</artifactId>
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-framework-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-parameter</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-expression-language</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <exclusions>
                 <exclusion>
                     <groupId>asm</groupId>
@@ -56,7 +56,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-data-provenance-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <!-- Dependency marked as compile, not test, because we have assertion
diff --git a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-nar/pom.xml b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-nar/pom.xml
index 5bcdf9f..20df28e 100644
--- a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-nar/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-accumulo-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-accumulo-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>false</maven.javadoc.skip>
@@ -34,17 +34,17 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-accumulo-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-accumulo-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/pom.xml b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/pom.xml
index bd2b9e3..c592237 100644
--- a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
 	    <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-accumulo-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-accumulo-processors</artifactId>
@@ -29,7 +29,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-accumulo-services-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
 
@@ -46,34 +46,34 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-serialization-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-path</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.slf4j</groupId>
@@ -100,7 +100,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-accumulo-services</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-services-api-nar/pom.xml b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-services-api-nar/pom.xml
index b98106d..aa58dba 100644
--- a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-services-api-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-services-api-nar/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-accumulo-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-accumulo-services-api-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>false</maven.javadoc.skip>
@@ -34,18 +34,18 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-accumulo-services-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-services-api/pom.xml b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-services-api/pom.xml
index daa9de7..561db37 100644
--- a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-services-api/pom.xml
+++ b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-services-api/pom.xml
@@ -19,7 +19,7 @@
     <parent>
 	    <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-accumulo-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-accumulo-services-api</artifactId>
@@ -39,22 +39,22 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-serialization-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-path</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
 
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-services-nar/pom.xml b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-services-nar/pom.xml
index 918f04d..c634da3 100644
--- a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-services-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-services-nar/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-accumulo-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-accumulo-services-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>false</maven.javadoc.skip>
@@ -34,18 +34,18 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-accumulo-services</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
            <artifactId>nifi-accumulo-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-services/pom.xml b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-services/pom.xml
index e38884e..cb550d0 100644
--- a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-services/pom.xml
+++ b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-services/pom.xml
@@ -19,7 +19,7 @@
     <parent>
 	    <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-accumulo-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-accumulo-services</artifactId>
@@ -39,32 +39,32 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-lookup-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-serialization-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-path</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-accumulo-services-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-accumulo-bundle/pom.xml b/nifi-nar-bundles/nifi-accumulo-bundle/pom.xml
index d19466c..3507b4a 100644
--- a/nifi-nar-bundles/nifi-accumulo-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-accumulo-bundle/pom.xml
@@ -15,7 +15,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <properties>
@@ -23,7 +23,7 @@
     </properties>
 
     <artifactId>nifi-accumulo-bundle</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <modules>
@@ -40,12 +40,12 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-accumulo-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-services</artifactId>
-                <version>1.12.0-SNAPSHOT</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-nar/pom.xml b/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-nar/pom.xml
index 58de5d9..d0ead39 100644
--- a/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-ambari-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-ambari-nar</artifactId>
@@ -30,7 +30,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-ambari-reporting-task</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/pom.xml b/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/pom.xml
index e2ac9b8..a820f79 100644
--- a/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/pom.xml
+++ b/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-ambari-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-ambari-reporting-task</artifactId>
@@ -41,23 +41,23 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-reporting-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-metrics</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <!-- test dependencies -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-ambari-bundle/pom.xml b/nifi-nar-bundles/nifi-ambari-bundle/pom.xml
index 35840dd..4b75f87 100644
--- a/nifi-nar-bundles/nifi-ambari-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-ambari-bundle/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-ambari-bundle</artifactId>
diff --git a/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-nar/pom.xml b/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-nar/pom.xml
index eb13d83..da716e5 100644
--- a/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-nar/pom.xml
@@ -18,10 +18,10 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-amqp-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-amqp-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -31,7 +31,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/pom.xml b/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/pom.xml
index 0f5530d..b4d1f06 100644
--- a/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/pom.xml
@@ -14,7 +14,7 @@ language governing permissions and limitations under the License. -->
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-amqp-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-amqp-processors</artifactId>
     <packaging>jar</packaging>
@@ -40,12 +40,12 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-amqp-bundle/pom.xml b/nifi-nar-bundles/nifi-amqp-bundle/pom.xml
index f23ed55..2e53b2d 100644
--- a/nifi-nar-bundles/nifi-amqp-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-amqp-bundle/pom.xml
@@ -18,10 +18,10 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-amqp-bundle</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>pom</packaging>
     <description>A bundle of processors that publish to and consume messages from AMQP.</description>
     <modules>
@@ -33,7 +33,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-amqp-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-nar/pom.xml b/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-nar/pom.xml
index 0d35816..d829f45 100644
--- a/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-nar/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-atlas-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-atlas-nar</artifactId>
@@ -37,7 +37,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/pom.xml b/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/pom.xml
index 1611724..c539d5a 100644
--- a/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/pom.xml
+++ b/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-atlas-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-atlas-reporting-task</artifactId>
@@ -33,12 +33,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-reporting-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -93,7 +93,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-atlas-bundle/pom.xml b/nifi-nar-bundles/nifi-atlas-bundle/pom.xml
index 9424bbf..9f3bf6d 100644
--- a/nifi-nar-bundles/nifi-atlas-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-atlas-bundle/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-atlas-bundle</artifactId>
@@ -39,7 +39,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-atlas-reporting-task</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
 
             <dependency>
diff --git a/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-nar/pom.xml b/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-nar/pom.xml
index 298b431..14fffa5 100644
--- a/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-nar/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-avro-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-avro-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -34,7 +34,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-avro-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/pom.xml b/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/pom.xml
index 54f3b4e..658c53f 100644
--- a/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/pom.xml
@@ -15,7 +15,7 @@ language governing permissions and limitations under the License. -->
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-avro-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-avro-processors</artifactId>
@@ -29,7 +29,7 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.commons</groupId>
@@ -54,7 +54,7 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-avro-bundle/pom.xml b/nifi-nar-bundles/nifi-avro-bundle/pom.xml
index 3d969eb..01f9cc0 100644
--- a/nifi-nar-bundles/nifi-avro-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-avro-bundle/pom.xml
@@ -19,12 +19,12 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-avro-bundle</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <modules>
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/pom.xml b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/pom.xml
index 61e079c..a32c512 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <artifactId>nifi-aws-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-aws-abstract-processors</artifactId>
@@ -80,7 +80,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-aws-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
@@ -90,7 +90,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-nar/pom.xml b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-nar/pom.xml
index 7b4d1dc..a611e95 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-nar/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-aws-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-aws-nar</artifactId>
@@ -33,13 +33,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-aws-service-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-aws-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml
index 88f06bc..0db840e 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-aws-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-aws-processors</artifactId>
@@ -32,29 +32,29 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-aws-abstract-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-aws-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-web-test-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-service-api-nar/pom.xml b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-service-api-nar/pom.xml
index 001e347..014073b 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-service-api-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-service-api-nar/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-aws-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-aws-service-api-nar</artifactId>
@@ -33,13 +33,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-aws-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-service-api/pom.xml b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-service-api/pom.xml
index b2d266c..92b7568 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-service-api/pom.xml
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-service-api/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <artifactId>nifi-aws-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/pom.xml b/nifi-nar-bundles/nifi-aws-bundle/pom.xml
index b6a5dc4..2794bda 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-aws-bundle/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-aws-bundle</artifactId>
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-nar/pom.xml b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-nar/pom.xml
index 024eeb4..1006ba4 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-nar/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-azure-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-azure-nar</artifactId>
@@ -33,17 +33,17 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-azure-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-azure-reporting-task</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-azure-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/pom.xml b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/pom.xml
index fffd311..1a4d9cd 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/pom.xml
@@ -15,7 +15,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-azure-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-azure-processors</artifactId>
     <packaging>jar</packaging>
@@ -31,12 +31,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-service-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -53,7 +53,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-azure-services-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
@@ -88,19 +88,19 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-reporting-task/pom.xml b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-reporting-task/pom.xml
index 1e71200..c4cf600 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-reporting-task/pom.xml
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-reporting-task/pom.xml
@@ -15,7 +15,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-azure-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-azure-reporting-task</artifactId>
     <packaging>jar</packaging>
@@ -25,17 +25,17 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-metrics</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>com.google.code.gson</groupId>
@@ -56,14 +56,14 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     
       <dependency>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-reporting-utils</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
       </dependency>
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api-nar/pom.xml b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api-nar/pom.xml
index 0e71114..429aa67 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <artifactId>nifi-azure-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
@@ -33,13 +33,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-azure-services-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/pom.xml b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/pom.xml
index 3ea5278..2b676e6 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/pom.xml
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <artifactId>nifi-azure-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
diff --git a/nifi-nar-bundles/nifi-azure-bundle/pom.xml b/nifi-nar-bundles/nifi-azure-bundle/pom.xml
index 6d61d92..cc5bd3a 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-azure-bundle/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-azure-bundle</artifactId>
diff --git a/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-nar/pom.xml b/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-nar/pom.xml
index 677a92f..66a881e 100644
--- a/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-nar/pom.xml
@@ -19,23 +19,23 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-beats-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-beats-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
 
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-beats-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-processors/pom.xml b/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-processors/pom.xml
index 4b65243..1afd66f 100644
--- a/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-beats-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-beats-processors</artifactId>
@@ -33,7 +33,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>com.google.code.gson</groupId>
@@ -43,17 +43,17 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-socket-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-flowfile-packager</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -63,7 +63,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-beats-bundle/pom.xml b/nifi-nar-bundles/nifi-beats-bundle/pom.xml
index 72892d8..c1a7362 100644
--- a/nifi-nar-bundles/nifi-beats-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-beats-bundle/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-beats-bundle</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <modules>
diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-nar/pom.xml b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-nar/pom.xml
index 7607312..70ca50f 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-nar/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-cassandra-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-cassandra-nar</artifactId>
@@ -33,7 +33,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-cassandra-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/pom.xml b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/pom.xml
index 1aac1b8..ea484e2 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-cassandra-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-cassandra-processors</artifactId>
@@ -33,7 +33,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -62,7 +62,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-cassandra-services</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-api-nar/pom.xml b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-api-nar/pom.xml
index 1e74c32..ee03967 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-api-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-api-nar/pom.xml
@@ -17,25 +17,25 @@
     <parent>
         <artifactId>nifi-cassandra-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>nifi-cassandra-services-api-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
 
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-cassandra-services-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-api/pom.xml b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-api/pom.xml
index 63f6848..9544c99 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-api/pom.xml
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-api/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <artifactId>nifi-cassandra-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-nar/pom.xml b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-nar/pom.xml
index 28a6e95..98f0379 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <artifactId>nifi-cassandra-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
@@ -28,13 +28,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-cassandra-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-cassandra-services</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services/pom.xml b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services/pom.xml
index 820cb37..2fe331d 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services/pom.xml
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <artifactId>nifi-cassandra-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
@@ -29,13 +29,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
 
         <dependency>
@@ -53,25 +53,25 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-ssl-context-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-framework-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
 
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/pom.xml b/nifi-nar-bundles/nifi-cassandra-bundle/pom.xml
index 0ff174b..30808f8 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <properties>
@@ -43,7 +43,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-cassandra-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-ccda-bundle/nifi-ccda-nar/pom.xml b/nifi-nar-bundles/nifi-ccda-bundle/nifi-ccda-nar/pom.xml
index cb9ec84..a9f005e 100644
--- a/nifi-nar-bundles/nifi-ccda-bundle/nifi-ccda-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-ccda-bundle/nifi-ccda-nar/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-ccda-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-ccda-nar</artifactId>
diff --git a/nifi-nar-bundles/nifi-ccda-bundle/nifi-ccda-processors/pom.xml b/nifi-nar-bundles/nifi-ccda-bundle/nifi-ccda-processors/pom.xml
index 1d62bfb..9fa8848 100644
--- a/nifi-nar-bundles/nifi-ccda-bundle/nifi-ccda-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-ccda-bundle/nifi-ccda-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-ccda-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-ccda-processors</artifactId>
@@ -33,7 +33,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.commons</groupId>
@@ -63,7 +63,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-ccda-bundle/pom.xml b/nifi-nar-bundles/nifi-ccda-bundle/pom.xml
index 7afc6f8..3997274 100644
--- a/nifi-nar-bundles/nifi-ccda-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-ccda-bundle/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-ccda-bundle</artifactId>
@@ -35,7 +35,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-ccda-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-cdc/nifi-cdc-api/pom.xml b/nifi-nar-bundles/nifi-cdc/nifi-cdc-api/pom.xml
index 9bb4820..498b453 100644
--- a/nifi-nar-bundles/nifi-cdc/nifi-cdc-api/pom.xml
+++ b/nifi-nar-bundles/nifi-cdc/nifi-cdc-api/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-cdc</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     
     <artifactId>nifi-cdc-api</artifactId>
diff --git a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-nar/pom.xml b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-nar/pom.xml
index 623dce1..1125906 100644
--- a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-cdc-mysql-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-cdc-mysql-nar</artifactId>
     <packaging>nar</packaging>
@@ -30,7 +30,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/pom.xml b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/pom.xml
index 62b6833..ce43a7a 100644
--- a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/pom.xml
@@ -14,7 +14,7 @@ language governing permissions and limitations under the License. -->
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-cdc-mysql-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-cdc-mysql-processors</artifactId>
     <packaging>jar</packaging>
@@ -27,12 +27,12 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-cdc-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -46,7 +46,7 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
@@ -62,7 +62,7 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-ssl-context-service</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/pom.xml b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/pom.xml
index aea76b2..1ea852c 100644
--- a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-cdc</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-cdc-mysql-bundle</artifactId>
     <packaging>pom</packaging>
@@ -32,7 +32,7 @@
           <dependency>
               <groupId>org.apache.nifi</groupId>
               <artifactId>nifi-cdc-mysql-processors</artifactId>
-              <version>1.12.0</version>
+              <version>1.12.1-SNAPSHOT</version>
           </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-cdc/pom.xml b/nifi-nar-bundles/nifi-cdc/pom.xml
index df45660..12904a3 100644
--- a/nifi-nar-bundles/nifi-cdc/pom.xml
+++ b/nifi-nar-bundles/nifi-cdc/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-cdc</artifactId>
     <packaging>pom</packaging>
diff --git a/nifi-nar-bundles/nifi-confluent-platform-bundle/nifi-confluent-platform-nar/pom.xml b/nifi-nar-bundles/nifi-confluent-platform-bundle/nifi-confluent-platform-nar/pom.xml
index eb84fcd..6d0d8ed 100644
--- a/nifi-nar-bundles/nifi-confluent-platform-bundle/nifi-confluent-platform-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-confluent-platform-bundle/nifi-confluent-platform-nar/pom.xml
@@ -14,7 +14,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-confluent-platform-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-confluent-platform-nar</artifactId>
     <packaging>nar</packaging>
@@ -22,13 +22,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-confluent-schema-registry-service</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/pom.xml b/nifi-nar-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/pom.xml
index 9893243..f3e0847 100644
--- a/nifi-nar-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/pom.xml
+++ b/nifi-nar-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/pom.xml
@@ -15,7 +15,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-confluent-platform-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-confluent-schema-registry-service</artifactId>
@@ -35,7 +35,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-avro-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -59,7 +59,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-web-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-confluent-platform-bundle/pom.xml b/nifi-nar-bundles/nifi-confluent-platform-bundle/pom.xml
index d2915f6..d8708a2 100644
--- a/nifi-nar-bundles/nifi-confluent-platform-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-confluent-platform-bundle/pom.xml
@@ -14,7 +14,7 @@
 	<parent>
 		<groupId>org.apache.nifi</groupId>
 		<artifactId>nifi-nar-bundles</artifactId>
-		<version>1.12.0</version>
+		<version>1.12.1-SNAPSHOT</version>
 	</parent>
 	<artifactId>nifi-confluent-platform-bundle</artifactId>
 	<packaging>pom</packaging>
diff --git a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-nar/pom.xml b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-nar/pom.xml
index 8de5186..881cd2a 100644
--- a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-nar/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-couchbase-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-couchbase-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -34,13 +34,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-couchbase-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-couchbase-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/pom.xml b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/pom.xml
index 90d8a2a..6fb990b 100644
--- a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-couchbase-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-couchbase-processors</artifactId>
@@ -33,12 +33,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-couchbase-services-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
@@ -65,7 +65,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-services-api-nar/pom.xml b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-services-api-nar/pom.xml
index 139a2ff..9b4a24e 100644
--- a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-services-api-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-services-api-nar/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-couchbase-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-couchbase-services-api-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -34,13 +34,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-couchbase-services-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-services-api/pom.xml b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-services-api/pom.xml
index 9a895c0..fcedcd0 100644
--- a/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-services-api/pom.xml
+++ b/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-services-api/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-couchbase-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-couchbase-services-api</artifactId>
diff --git a/nifi-nar-bundles/nifi-couchbase-bundle/pom.xml b/nifi-nar-bundles/nifi-couchbase-bundle/pom.xml
index f483bc5..1cd65e8 100644
--- a/nifi-nar-bundles/nifi-couchbase-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-couchbase-bundle/pom.xml
@@ -19,12 +19,12 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-couchbase-bundle</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <modules>
diff --git a/nifi-nar-bundles/nifi-cybersecurity-bundle/nifi-cybersecurity-nar/pom.xml b/nifi-nar-bundles/nifi-cybersecurity-bundle/nifi-cybersecurity-nar/pom.xml
index 07f0f62..ecb821c 100644
--- a/nifi-nar-bundles/nifi-cybersecurity-bundle/nifi-cybersecurity-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-cybersecurity-bundle/nifi-cybersecurity-nar/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-cybersecurity-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-cybersecurity-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -34,7 +34,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-cybersecurity-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-cybersecurity-bundle/nifi-cybersecurity-processors/pom.xml b/nifi-nar-bundles/nifi-cybersecurity-bundle/nifi-cybersecurity-processors/pom.xml
index 2a269a0..65db385 100644
--- a/nifi-nar-bundles/nifi-cybersecurity-bundle/nifi-cybersecurity-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-cybersecurity-bundle/nifi-cybersecurity-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-cybersecurity-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-cybersecurity-processors</artifactId>
@@ -33,7 +33,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>info.debatty</groupId>
@@ -48,7 +48,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-cybersecurity-bundle/pom.xml b/nifi-nar-bundles/nifi-cybersecurity-bundle/pom.xml
index bbc54fe..7402e41 100644
--- a/nifi-nar-bundles/nifi-cybersecurity-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-cybersecurity-bundle/pom.xml
@@ -19,12 +19,12 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-cybersecurity-bundle</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <modules>
diff --git a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-nar/pom.xml b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-nar/pom.xml
index d332e39..caa9ffd 100644
--- a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-datadog-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-datadog-nar</artifactId>
diff --git a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/pom.xml b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/pom.xml
index 437d349..f93621c 100644
--- a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/pom.xml
+++ b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-datadog-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-datadog-reporting-task</artifactId>
@@ -46,12 +46,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-metrics</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.coursera</groupId>
@@ -78,7 +78,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-datadog-bundle/pom.xml b/nifi-nar-bundles/nifi-datadog-bundle/pom.xml
index f121b6b..8ab7a2f 100644
--- a/nifi-nar-bundles/nifi-datadog-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-datadog-bundle/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-datadog-bundle</artifactId>
@@ -34,7 +34,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-datadog-reporting-task</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.glassfish.jersey.core</groupId>
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service-api-nar/pom.xml b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service-api-nar/pom.xml
index d4f65d4..7153446 100644
--- a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service-api-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service-api-nar/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-druid-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-druid-controller-service-api-nar</artifactId>
@@ -28,13 +28,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-druid-controller-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service-api/pom.xml b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service-api/pom.xml
index c5f3f8e..cc0aa51 100644
--- a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service-api/pom.xml
+++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service-api/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-druid-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-druid-controller-service-api</artifactId>
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/pom.xml b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/pom.xml
index 51646ea..72ccc5c 100644
--- a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/pom.xml
+++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-druid-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-druid-controller-service</artifactId>
@@ -33,12 +33,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-druid-controller-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
@@ -59,7 +59,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-nar/pom.xml b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-nar/pom.xml
index 9854740..bf95022 100644
--- a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-nar/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-druid-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
   
     <artifactId>nifi-druid-nar</artifactId>
@@ -28,18 +28,18 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-druid-controller-service-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-druid-controller-service</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-druid-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-processors/pom.xml b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-processors/pom.xml
index bfddc98..460e5ed 100644
--- a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-processors/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-druid-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-druid-processors</artifactId>
@@ -32,7 +32,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -45,7 +45,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-druid-controller-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
@@ -56,19 +56,19 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-druid-controller-service</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-druid-bundle/pom.xml b/nifi-nar-bundles/nifi-druid-bundle/pom.xml
index 0170e15..5bb0e22 100644
--- a/nifi-nar-bundles/nifi-druid-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-druid-bundle/pom.xml
@@ -18,11 +18,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-druid-bundle</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>
diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-nar/pom.xml b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-nar/pom.xml
index f3c14f0..a03b946 100644
--- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-nar/pom.xml
@@ -17,12 +17,12 @@
     <parent>
         <artifactId>nifi-easyrules-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>nifi-easyrules-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -32,12 +32,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-easyrules-service</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/pom.xml b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/pom.xml
index 0228903..123c21b 100644
--- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/pom.xml
+++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <artifactId>nifi-easyrules-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
@@ -27,7 +27,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-rules-engine-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.yaml</groupId>
@@ -37,7 +37,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.jeasy</groupId>
@@ -68,7 +68,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/pom.xml b/nifi-nar-bundles/nifi-easyrules-bundle/pom.xml
index a2d8b70..7df5b57 100644
--- a/nifi-nar-bundles/nifi-easyrules-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-easyrules-bundle/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <artifactId>nifi-nar-bundles</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
@@ -33,7 +33,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-easyrules-service</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-nar/pom.xml b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-nar/pom.xml
index 0de6cc3..25a2f5a 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-nar/pom.xml
@@ -14,7 +14,7 @@ language governing permissions and limitations under the License. -->
     <parent>
         <artifactId>nifi-elasticsearch-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.nifi</groupId>
@@ -30,7 +30,7 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/pom.xml b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/pom.xml
index 7506cd8..5d2f1d7 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/pom.xml
@@ -14,7 +14,7 @@ language governing permissions and limitations under the License. -->
     <parent>
         <artifactId>nifi-elasticsearch-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-elasticsearch-5-processors</artifactId>
@@ -40,12 +40,12 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api-nar/pom.xml b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api-nar/pom.xml
index 2a0ed09..1dbe8a9 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api-nar/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-elasticsearch-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-elasticsearch-client-service-api-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -34,13 +34,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-elasticsearch-client-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api/pom.xml b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api/pom.xml
index 7bb2ab6..f16854d 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api/pom.xml
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-elasticsearch-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-elasticsearch-client-service-api</artifactId>
@@ -29,18 +29,18 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-ssl-context-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-nar/pom.xml b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-nar/pom.xml
index 7c41d2e..7e7d935 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-nar/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-elasticsearch-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-elasticsearch-client-service-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -34,13 +34,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-elasticsearch-client-service-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-elasticsearch-client-service</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml
index 05778a2..4c05a8f 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-elasticsearch-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-elasticsearch-client-service</artifactId>
@@ -36,29 +36,29 @@
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-lookup-service-api</artifactId>
             <scope>provided</scope>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
             <scope>provided</scope>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-distributed-cache-client-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
 
@@ -94,19 +94,19 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-ssl-context-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-elasticsearch-client-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
@@ -130,7 +130,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-avro-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
         <dependency>
@@ -151,7 +151,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-path</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-nar/pom.xml b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-nar/pom.xml
index fab65dc..25b98d5 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-nar/pom.xml
@@ -14,7 +14,7 @@ language governing permissions and limitations under the License. -->
     <parent>
         <artifactId>nifi-elasticsearch-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.nifi</groupId>
@@ -30,7 +30,7 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/pom.xml b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/pom.xml
index babf441..4782ca4 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/pom.xml
@@ -14,7 +14,7 @@ language governing permissions and limitations under the License. -->
     <parent>
         <artifactId>nifi-elasticsearch-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-elasticsearch-processors</artifactId>
@@ -40,12 +40,12 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-path</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -72,13 +72,13 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
@@ -124,12 +124,12 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-web-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-nar/pom.xml b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-nar/pom.xml
index fef4443..a902c41 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-nar/pom.xml
@@ -14,7 +14,7 @@ language governing permissions and limitations under the License. -->
     <parent>
         <artifactId>nifi-elasticsearch-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.nifi</groupId>
@@ -30,13 +30,13 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-elasticsearch-client-service-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-elasticsearch-restapi-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-processors/pom.xml b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-processors/pom.xml
index 134b04a..4a1b688 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-processors/pom.xml
@@ -14,7 +14,7 @@ language governing permissions and limitations under the License. -->
     <parent>
         <artifactId>nifi-elasticsearch-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-elasticsearch-restapi-processors</artifactId>
@@ -30,36 +30,36 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-properties</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-ssl-context-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-schema-registry-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>commons-io</groupId>
@@ -79,7 +79,7 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-ssl-context-service</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
@@ -90,13 +90,13 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-elasticsearch-client-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-json-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
         <dependency>
@@ -118,19 +118,19 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-path</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-serialization-services</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/pom.xml b/nifi-nar-bundles/nifi-elasticsearch-bundle/pom.xml
index 7f1e831..9ea38f4 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/pom.xml
@@ -15,7 +15,7 @@ language governing permissions and limitations under the License. -->
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.nifi</groupId>
@@ -40,17 +40,17 @@ language governing permissions and limitations under the License. -->
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-elasticsearch-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-elasticsearch-5-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-elasticsearch-restapi-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-nar/pom.xml b/nifi-nar-bundles/nifi-email-bundle/nifi-email-nar/pom.xml
index 2b9082b..ba982cc 100644
--- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-nar/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-email-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-email-nar</artifactId>
@@ -33,7 +33,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/pom.xml b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/pom.xml
index 6bcda7f..2fbae41 100644
--- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-email-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-email-processors</artifactId>
@@ -33,7 +33,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>javax.mail</groupId>
@@ -114,7 +114,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-email-bundle/pom.xml b/nifi-nar-bundles/nifi-email-bundle/pom.xml
index 66652ae..c71ad0d 100644
--- a/nifi-nar-bundles/nifi-email-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-email-bundle/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-email-bundle</artifactId>
@@ -35,7 +35,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-email-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-nar/pom.xml b/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-nar/pom.xml
index 357979b..75b5a0d 100644
--- a/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-nar/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-enrich-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-enrich-nar</artifactId>
     <packaging>nar</packaging>
diff --git a/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/pom.xml b/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/pom.xml
index 4ad911f..b10992b 100644
--- a/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-enrich-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
 
@@ -37,7 +37,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>com.maxmind.geoip2</groupId>
@@ -58,7 +58,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
@@ -138,19 +138,19 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-path</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-serialization-services</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-enrich-bundle/pom.xml b/nifi-nar-bundles/nifi-enrich-bundle/pom.xml
index b3d4dc2..6e434ec 100644
--- a/nifi-nar-bundles/nifi-enrich-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-enrich-bundle/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-enrich-bundle</artifactId>
@@ -35,7 +35,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-enrich-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-nar/pom.xml b/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-nar/pom.xml
index 306f376..33f4b55 100644
--- a/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-nar/pom.xml
@@ -14,7 +14,7 @@ language governing permissions and limitations under the License. -->
     <parent>
         <artifactId>nifi-evtx-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.nifi</groupId>
@@ -29,7 +29,7 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-processors/pom.xml b/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-processors/pom.xml
index 5aecc44..e92a642 100644
--- a/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-processors/pom.xml
@@ -20,7 +20,7 @@
     <parent>
         <artifactId>nifi-evtx-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-evtx-processors</artifactId>
@@ -40,7 +40,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>com.google.guava</groupId>
@@ -55,7 +55,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-evtx-bundle/pom.xml b/nifi-nar-bundles/nifi-evtx-bundle/pom.xml
index da8efb2..32875b0 100644
--- a/nifi-nar-bundles/nifi-evtx-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-evtx-bundle/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.nifi</groupId>
@@ -43,7 +43,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-evtx-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-database-test-utils/pom.xml b/nifi-nar-bundles/nifi-extension-utils/nifi-database-test-utils/pom.xml
index 847e01c..97efe06 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-database-test-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-database-test-utils/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-extension-utils</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-database-test-utils</artifactId>
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/pom.xml b/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/pom.xml
index 67939a5..3c6a691 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-extension-utils</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-database-utils</artifactId>
@@ -27,12 +27,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-avro-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>com.github.ben-manes.caffeine</groupId>
@@ -57,7 +57,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/pom.xml b/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/pom.xml
index 7826935..8cde789 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/pom.xml
@@ -18,10 +18,10 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-extension-utils</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-hadoop-utils</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>jar</packaging>
 
     <dependencies>
@@ -32,12 +32,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/pom.xml b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/pom.xml
index 4e29d40..2670045 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-extension-utils</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-processor-utils</artifactId>
     <packaging>jar</packaging>
@@ -38,12 +38,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>commons-io</groupId>
@@ -69,7 +69,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-prometheus-utils/pom.xml b/nifi-nar-bundles/nifi-extension-utils/nifi-prometheus-utils/pom.xml
index 9048667..a16948d 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-prometheus-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-prometheus-utils/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-extension-utils</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-prometheus-utils</artifactId>
     <packaging>jar</packaging>
@@ -41,17 +41,17 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-properties</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-metrics</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <!-- The client -->
         <dependency>
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/pom.xml b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/pom.xml
index 40d4b71..f5a8eb6 100755
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-record-utils</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     
     <artifactId>nifi-avro-record-utils</artifactId>
@@ -27,7 +27,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.avro</groupId>
@@ -60,7 +60,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-hadoop-record-utils/pom.xml b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-hadoop-record-utils/pom.xml
index fdd49b4..e5513b0 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-hadoop-record-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-hadoop-record-utils/pom.xml
@@ -18,22 +18,22 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-record-utils</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-hadoop-record-utils</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>jar</packaging>
 
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hadoop-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-avro-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <!-- Other modules using nifi-hadoop-utils are expected to have the below dependencies available, typically through a NAR dependency -->
         <dependency>
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/pom.xml b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/pom.xml
index ecbcdc3..ee472d1 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-record-utils</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     
     <artifactId>nifi-mock-record-utils</artifactId>
@@ -31,7 +31,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <!-- Other modules using nifi-standard-record-utils are expected to have this API available, typically through a NAR dependency -->
         <dependency>
@@ -49,7 +49,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-avro-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/pom.xml b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/pom.xml
index 9261a95..3311ca5 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-record-utils</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     
     <artifactId>nifi-standard-record-utils</artifactId>
@@ -31,12 +31,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <!-- Other modules using nifi-standard-record-utils are expected to have these APIs available, typically through a NAR dependency -->
         <dependency>
@@ -64,7 +64,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
 	    <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/pom.xml b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/pom.xml
index 00ecd87..d0696aa 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-extension-utils</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <packaging>pom</packaging>
     <artifactId>nifi-record-utils</artifactId>
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-reporting-utils/pom.xml b/nifi-nar-bundles/nifi-extension-utils/nifi-reporting-utils/pom.xml
index 7b83c16..6764237 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-reporting-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-reporting-utils/pom.xml
@@ -17,12 +17,12 @@
     <parent>
         <artifactId>nifi-extension-utils</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>nifi-reporting-utils</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>jar</packaging>
 
     <dependencies>
@@ -33,7 +33,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.commons</groupId>
@@ -48,7 +48,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-metrics</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-service-utils/pom.xml b/nifi-nar-bundles/nifi-extension-utils/nifi-service-utils/pom.xml
index e8a1ea3..bb2cff6 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-service-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-service-utils/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-extension-utils</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-service-utils</artifactId>
     <packaging>jar</packaging>
@@ -38,12 +38,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-syslog-utils/pom.xml b/nifi-nar-bundles/nifi-extension-utils/nifi-syslog-utils/pom.xml
index ef42d14..c3a6ee0 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-syslog-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-syslog-utils/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-extension-utils</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-syslog-utils</artifactId>
     <packaging>jar</packaging>
@@ -31,12 +31,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-extension-utils/pom.xml b/nifi-nar-bundles/nifi-extension-utils/pom.xml
index bc304ca..2980f23 100644
--- a/nifi-nar-bundles/nifi-extension-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-extension-utils/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <packaging>pom</packaging>
     <artifactId>nifi-extension-utils</artifactId>
diff --git a/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml b/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
index c7865e8..d3fca47 100644
--- a/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
@@ -18,10 +18,10 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-flume-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-flume-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -130,7 +130,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hadoop-libraries-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml b/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
index 51a58e6..4dae245 100644
--- a/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-flume-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-flume-processors</artifactId>
     <packaging>jar</packaging>
@@ -35,12 +35,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-flowfile-packager</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.flume</groupId>
@@ -154,7 +154,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-flume-bundle/pom.xml b/nifi-nar-bundles/nifi-flume-bundle/pom.xml
index f844c56..8c4d610 100644
--- a/nifi-nar-bundles/nifi-flume-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-flume-bundle/pom.xml
@@ -18,10 +18,10 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-flume-bundle</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>pom</packaging>
     <description>A bundle of processors that run Flume sources/sinks</description>
     <modules>
@@ -33,7 +33,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-flume-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/pom.xml
index 0f575f1..55c9198 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-framework-nar</artifactId>
     <packaging>nar</packaging>
@@ -46,12 +46,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-prioritizers</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-stateless</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
 
         <!-- mark these nifi artifacts as provided since it is included in the lib -->
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/pom.xml
index 8987cd6..748a5d1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-administration</artifactId>
     <dependencies>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/pom.xml
index d43588a..7f42c4e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <artifactId>nifi-framework</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
@@ -157,7 +157,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock-authorizer</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/pom.xml
index 28af46b..3419a75 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-client-dto</artifactId>
     <dependencies>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml
index c3e3f0b..76a91a4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml
@@ -14,7 +14,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-documentation</artifactId>
     <dependencies>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/pom.xml
index 1e475f7..ebbeeee 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-file-authorizer</artifactId>
     <build>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/pom.xml
index 43a5c30..3b7a215 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/pom.xml
@@ -14,7 +14,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-flowfile-repo-serialization</artifactId>
 
@@ -30,17 +30,17 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-repository-models</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-write-ahead-log</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-schema-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/pom.xml
index 0e9e46e..2a0c8a4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-framework-authorization</artifactId>
 
@@ -34,7 +34,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -55,7 +55,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock-authorizer</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/pom.xml
index abe2810..203658e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-framework-cluster-protocol</artifactId>
     <packaging>jar</packaging>
@@ -36,12 +36,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-logging-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-socket-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/pom.xml
index 813b199..d31384c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/pom.xml
@@ -14,7 +14,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-framework-cluster</artifactId>
     <packaging>jar</packaging>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml
index 93339d9..b9dd9fd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml
@@ -14,7 +14,7 @@ language governing permissions and limitations under the License. -->
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-framework-core-api</artifactId>
     <dependencies>
@@ -29,7 +29,7 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -50,7 +50,7 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-site-to-site-client</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.commons</groupId>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
index 025b61d..a04a729 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-framework-core</artifactId>
     <packaging>jar</packaging>
@@ -46,22 +46,22 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-expression-language</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-schema-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-repository-models</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -82,7 +82,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-logging-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -91,7 +91,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-site-to-site-client</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -137,22 +137,22 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-data-provenance-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-write-ahead-log</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-rocksdb-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-flowfile-repo-serialization</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.zookeeper</groupId>
@@ -213,7 +213,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
@@ -238,13 +238,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock-authorizer</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-persistent-provenance-repository</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/pom.xml
index a73073f..ced4766 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-framework-nar-loading-utils</artifactId>
     <packaging>jar</packaging>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/pom.xml
index da6243e..1641995 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-framework-nar-utils</artifactId>
     <packaging>jar</packaging>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-mock-authorizer/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-mock-authorizer/pom.xml
index 6a269b0..447e5f7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-mock-authorizer/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-mock-authorizer/pom.xml
@@ -13,7 +13,7 @@
     <parent>
         <artifactId>nifi-framework</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/pom.xml
index 06dcfbd..6e255ad 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-nar-utils</artifactId>
     <packaging>jar</packaging>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/pom.xml
index 113e12c..e1ce1c2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-properties-loader</artifactId>
     <name>nifi-properties-loader</name>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/pom.xml
index a693d00..886678d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/pom.xml
@@ -14,7 +14,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-repository-models</artifactId>
 
@@ -30,7 +30,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-schema-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.commons</groupId>
@@ -39,7 +39,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
index 6f6246b..1b3c7f9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-resources</artifactId>
     <packaging>pom</packaging>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/pom.xml
index 2ef7698..08d018a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-runtime</artifactId>
     <packaging>jar</packaging>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-shell-authorizer/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-shell-authorizer/pom.xml
index a3ebef2..655e2c1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-shell-authorizer/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-shell-authorizer/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <artifactId>nifi-framework</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <artifactId>nifi-shell-authorizer</artifactId>
@@ -40,7 +40,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml
index e06c50d..fc94caa 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml
@@ -18,14 +18,14 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-site-to-site</artifactId>
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -58,17 +58,17 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-web-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-site-to-site-client</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.httpcomponents</groupId>
@@ -77,7 +77,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-standard-prioritizers/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-standard-prioritizers/pom.xml
index 7b679dc..19fd9dd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-standard-prioritizers/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-standard-prioritizers/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-standard-prioritizers</artifactId>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/pom.xml
index 0215d3c..27958dd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-stateless</artifactId>
@@ -28,47 +28,47 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-framework-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-framework-core-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-framework-nar-loading-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-nar-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-expression-language</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-data-provenance-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-parameter</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>com.google.code.gson</groupId>
@@ -108,13 +108,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-web-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>war</type>
             <scope>test</scope>
         </dependency>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/pom.xml
index 0e9746d..fe62ebb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-user-actions</artifactId>
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/pom.xml
index ca060a6..a880f6d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-web</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-custom-ui-utilities</artifactId>
     <dependencies>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml
index ca983cb..c864f80 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-web</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-jetty</artifactId>
     <packaging>jar</packaging>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/pom.xml
index ed0ff9e..2c7bdb1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-web</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-ui-extension</artifactId>
     <packaging>jar</packaging>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/pom.xml
index 26c5836..344ca6f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-web</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-web-api</artifactId>
@@ -202,7 +202,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-web-security</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>test-jar</type>
             <scope>test</scope>
         </dependency>
@@ -259,7 +259,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-prometheus-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>javax.servlet</groupId>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/pom.xml
index 81a0b19..070fe11 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-web</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-web-content-access</artifactId>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/pom.xml
index 97f66b2..ae229be 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-web</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-web-content-viewer</artifactId>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml
index 712e0a9..275ed59 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-web</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-web-docs</artifactId>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/pom.xml
index d6f08a4..23d9eaf 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-web</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-web-error</artifactId>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/pom.xml
index 3c6527c..930a20d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-web</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-web-optimistic-locking</artifactId>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/pom.xml
index 1c382d4..381f542 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-web</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-web-security</artifactId>
     <build>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
index 552518b..ac2efc7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-web</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-web-ui</artifactId>
     <packaging>war</packaging>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/pom.xml
index 27d5285..c05fd53 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-web</artifactId>
     <packaging>pom</packaging>
@@ -40,31 +40,31 @@
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-web-api</artifactId>
                 <type>war</type>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-web-error</artifactId>
                 <type>war</type>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-web-docs</artifactId>
                 <type>war</type>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-web-content-viewer</artifactId>
                 <type>war</type>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-web-ui</artifactId>
                 <type>war</type>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
index c24a3a7..80821a8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-framework-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-framework</artifactId>
     <packaging>pom</packaging>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/pom.xml
index 2c2a0f4..0c54ef6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-framework-bundle</artifactId>
     <packaging>pom</packaging>
@@ -36,168 +36,168 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-custom-ui-utilities</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-ui-extension</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-data-provenance-utils</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-security-utils</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-expression-language</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-mock</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
                 <scope>test</scope>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-web-utils</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-site-to-site-client</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-logging-utils</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-utils</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-framework-cluster-protocol</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-framework-cluster</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-runtime</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-client-dto</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-web-content-access</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-security</artifactId>
-                <version>1.12.0-SNAPSHOT</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-framework-core-api</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-framework-nar-utils</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-framework-nar-loading-utils</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-site-to-site</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-framework-core</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-properties-loader</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-framework-authorization</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-user-actions</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-administration</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-jetty</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-web-optimistic-locking</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-web-security</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-documentation</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-file-authorizer</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-shell-authorizer</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-authorizer</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-parameter</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>com.ibm.icu</groupId>
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-nar/pom.xml b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-nar/pom.xml
index ee49115..c47a5b1 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-nar/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-gcp-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-gcp-nar</artifactId>
@@ -33,13 +33,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-gcp-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-gcp-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml
index 43d4eb4..2d3e254 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-gcp-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-gcp-processors</artifactId>
@@ -53,18 +53,18 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-gcp-services-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
@@ -97,13 +97,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-json-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-serialization-services</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api-nar/pom.xml b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api-nar/pom.xml
index de8d16a..26061ec 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api-nar/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-gcp-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-gcp-services-api-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -34,13 +34,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-gcp-services-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api/pom.xml b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api/pom.xml
index bcf55ba..c57fa15 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api/pom.xml
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-gcp-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-gcp-services-api</artifactId>
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/pom.xml b/nifi-nar-bundles/nifi-gcp-bundle/pom.xml
index 4cbb36d..41646ea 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-gcp-bundle/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-gcp-bundle</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api-nar/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api-nar/pom.xml
index 8631c89..13db1e7 100644
--- a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api-nar/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-graph-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-graph-client-service-api-nar</artifactId>
@@ -29,13 +29,13 @@
 	    <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
 	        <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-graph-client-service-api</artifactId>
-	        <version>1.12.0</version>
+	        <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/pom.xml
index 4f38c0d..46e31c8 100644
--- a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/pom.xml
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-graph-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-graph-client-service-api</artifactId>
@@ -34,7 +34,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-nar/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-nar/pom.xml
index 9100249..1d1cfc4 100644
--- a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-nar/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-graph-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-graph-nar</artifactId>
@@ -33,7 +33,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-graph-client-service-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/pom.xml
index ae41e45..4974c9a 100644
--- a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-graph-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-graph-processors</artifactId>
@@ -37,7 +37,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>commons-io</groupId>
@@ -57,7 +57,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
@@ -69,13 +69,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-graph-client-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-graph-test-clients</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-test-clients/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-test-clients/pom.xml
index 1b6b2c6..d93780a 100644
--- a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-test-clients/pom.xml
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-test-clients/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-graph-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-graph-test-clients</artifactId>
@@ -34,7 +34,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -44,7 +44,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-graph-client-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
 
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service-nar/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service-nar/pom.xml
index 366b166..8f20224 100644
--- a/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-graph-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-neo4j-cypher-service-nar</artifactId>
     <packaging>nar</packaging>
@@ -25,13 +25,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-graph-client-service-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-neo4j-cypher-service</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/pom.xml
index 5828b04..0518dbb 100644
--- a/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/pom.xml
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/pom.xml
@@ -13,7 +13,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-graph-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-neo4j-cypher-service</artifactId>
     <packaging>jar</packaging>
@@ -26,7 +26,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -47,7 +47,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -56,13 +56,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-avro-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
         <dependency>
@@ -73,7 +73,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-json-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services-nar/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services-nar/pom.xml
index 5c3a08e..d722f13 100644
--- a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-graph-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-other-graph-services-nar</artifactId>
     <packaging>nar</packaging>
@@ -25,13 +25,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-graph-client-service-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-other-graph-services</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/pom.xml
index a01c613..a55743b 100644
--- a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/pom.xml
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/pom.xml
@@ -13,7 +13,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-graph-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-other-graph-services</artifactId>
     <packaging>jar</packaging>
@@ -26,7 +26,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -47,7 +47,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -56,13 +56,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-avro-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
         <dependency>
@@ -73,7 +73,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-json-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-graph-bundle/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/pom.xml
index 9deba1f..5816efa 100644
--- a/nifi-nar-bundles/nifi-graph-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-graph-bundle/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-graph-bundle</artifactId>
@@ -42,7 +42,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-graph-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-nar/pom.xml b/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-nar/pom.xml
index 8011b4a..715a664 100644
--- a/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-nar/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-groovyx-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-groovyx-nar</artifactId>
@@ -33,7 +33,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/pom.xml b/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/pom.xml
index a171ecf..7110ed0 100644
--- a/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-groovyx-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-groovyx-processors</artifactId>
@@ -34,12 +34,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-serialization-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.codehaus.groovy</groupId>
@@ -67,25 +67,25 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-dbcp-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-groovyx-bundle/pom.xml b/nifi-nar-bundles/nifi-groovyx-bundle/pom.xml
index 4524958..bc21618 100644
--- a/nifi-nar-bundles/nifi-groovyx-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-groovyx-bundle/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-groovyx-bundle</artifactId>
@@ -39,7 +39,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-groovyx-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.codehaus.groovy</groupId>
diff --git a/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-nar/pom.xml b/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-nar/pom.xml
index 740aa0a..41967a5 100644
--- a/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-nar/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-grpc-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-grpc-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -34,13 +34,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-grpc-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/pom.xml b/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/pom.xml
index e3583e6..116ea9e 100644
--- a/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/pom.xml
@@ -15,7 +15,7 @@ language governing permissions and limitations under the License. -->
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-grpc-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-grpc-processors</artifactId>
@@ -29,7 +29,7 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -68,7 +68,7 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-grpc-bundle/pom.xml b/nifi-nar-bundles/nifi-grpc-bundle/pom.xml
index 466bc98..aee1294 100644
--- a/nifi-nar-bundles/nifi-grpc-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-grpc-bundle/pom.xml
@@ -19,12 +19,12 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-grpc-bundle</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>pom</packaging>
     <description>A bundle of processors that speak the gRPC protocol</description>
 
@@ -38,7 +38,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-grpc-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml
index 7a185d6..0caff04 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-hadoop-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-hadoop-nar</artifactId>
     <packaging>nar</packaging>
@@ -29,7 +29,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hadoop-libraries-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml
index 85d26a7..3b26477 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-hadoop-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-hdfs-processors</artifactId>
     <packaging>jar</packaging>
@@ -30,17 +30,17 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hadoop-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-flowfile-packager</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.hadoop</groupId>
@@ -71,7 +71,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/pom.xml b/nifi-nar-bundles/nifi-hadoop-bundle/pom.xml
index cfd1314..6565271 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-hadoop-bundle</artifactId>
     <packaging>pom</packaging>
@@ -31,7 +31,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-hdfs-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-hadoop-libraries-bundle/nifi-hadoop-libraries-nar/pom.xml b/nifi-nar-bundles/nifi-hadoop-libraries-bundle/nifi-hadoop-libraries-nar/pom.xml
index 9c48e06..b4c6b2c 100644
--- a/nifi-nar-bundles/nifi-hadoop-libraries-bundle/nifi-hadoop-libraries-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-hadoop-libraries-bundle/nifi-hadoop-libraries-nar/pom.xml
@@ -13,7 +13,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-hadoop-libraries-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-hadoop-libraries-nar</artifactId>
     <packaging>nar</packaging>
@@ -26,7 +26,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-hadoop-libraries-bundle/pom.xml b/nifi-nar-bundles/nifi-hadoop-libraries-bundle/pom.xml
index ee176b7..c23e85d 100644
--- a/nifi-nar-bundles/nifi-hadoop-libraries-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-hadoop-libraries-bundle/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-hadoop-libraries-bundle</artifactId>
diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-nar/pom.xml b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-nar/pom.xml
index 3fe3e54..988677d 100644
--- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-nar/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-hbase-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-hbase-nar</artifactId>
     <packaging>nar</packaging>
@@ -30,7 +30,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/pom.xml b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/pom.xml
index 9a582f7..c5d8a95 100644
--- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-hbase-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-hbase-processors</artifactId>
     <description>Support for interacting with HBase</description>
@@ -26,7 +26,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hbase-client-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
@@ -41,7 +41,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -50,7 +50,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.commons</groupId>
@@ -70,20 +70,20 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-path</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.commons</groupId>
diff --git a/nifi-nar-bundles/nifi-hbase-bundle/pom.xml b/nifi-nar-bundles/nifi-hbase-bundle/pom.xml
index 5bf2d6b..6508b78 100644
--- a/nifi-nar-bundles/nifi-hbase-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-hbase-bundle/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-hbase-bundle</artifactId>
@@ -35,7 +35,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-hbase-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-nar/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-nar/pom.xml
index fef571c..9143249 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-nar/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-hive-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-hive-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -36,13 +36,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hive-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hive-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/pom.xml
index 49c652f..ac721ce 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-hive-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-hive-processors</artifactId>
@@ -34,23 +34,23 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-dbcp-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hive-services-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
@@ -99,7 +99,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hadoop-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>com.github.stephenc.findbugs</groupId>
@@ -114,7 +114,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api-nar/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api-nar/pom.xml
index dbe5973..1334a91 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api-nar/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-hive-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-hive-services-api-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -36,13 +36,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hive-services-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api/pom.xml
index 86bdc69..09d3011 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-hive-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-hive-services-api</artifactId>
@@ -34,12 +34,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-dbcp-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-nar/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-nar/pom.xml
index 0d59b73..46b577b 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-nar/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-hive-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-hive3-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -34,13 +34,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hive-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hive3-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml
index bb80b8d..edd36f8 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-hive-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-hive3-processors</artifactId>
@@ -29,23 +29,23 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-dbcp-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hive-services-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
@@ -55,7 +55,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hadoop-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -100,7 +100,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hadoop-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>com.github.stephenc.findbugs</groupId>
@@ -110,13 +110,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
@@ -132,13 +132,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-schema-registry-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-serialization-services</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
             <exclusions>
                 <exclusion>
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive_1_1-nar/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive_1_1-nar/pom.xml
index 7c41086..194a3b2 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive_1_1-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive_1_1-nar/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-hive-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-hive_1_1-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -36,13 +36,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hive-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hive_1_1-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive_1_1-processors/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive_1_1-processors/pom.xml
index 6c66b5b..21236f6 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive_1_1-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive_1_1-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-hive-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-hive_1_1-processors</artifactId>
@@ -34,23 +34,23 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-dbcp-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hive-services-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
@@ -99,7 +99,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hadoop-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>com.github.stephenc.findbugs</groupId>
@@ -114,7 +114,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-hive-bundle/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/pom.xml
index f81814e..0a8d361 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-hive-bundle</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <modules>
diff --git a/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-nar/pom.xml b/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-nar/pom.xml
index b8e4770..4f9243c 100644
--- a/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-nar/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-hl7-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-hl7-nar</artifactId>
@@ -33,7 +33,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hl7-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/pom.xml b/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/pom.xml
index 5fbacc5..2e4724f 100644
--- a/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-hl7-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-hl7-processors</artifactId>
@@ -47,7 +47,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.commons</groupId>
@@ -111,7 +111,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-hl7-bundle/pom.xml b/nifi-nar-bundles/nifi-hl7-bundle/pom.xml
index b2d3742..33152fc 100644
--- a/nifi-nar-bundles/nifi-hl7-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-hl7-bundle/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-hl7-bundle</artifactId>
@@ -35,12 +35,12 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-hl7-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-hl7-query-language</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-html-bundle/nifi-html-nar/pom.xml b/nifi-nar-bundles/nifi-html-bundle/nifi-html-nar/pom.xml
index 6d76464..736bd16 100644
--- a/nifi-nar-bundles/nifi-html-bundle/nifi-html-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-html-bundle/nifi-html-nar/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-html-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-html-nar</artifactId>
@@ -33,7 +33,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/pom.xml b/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/pom.xml
index 7f206b3..7ae404f 100644
--- a/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-html-bundle/nifi-html-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-html-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-html-processors</artifactId>
@@ -43,12 +43,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-html-bundle/pom.xml b/nifi-nar-bundles/nifi-html-bundle/pom.xml
index 8ca3f69..1ce8fc8 100644
--- a/nifi-nar-bundles/nifi-html-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-html-bundle/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-html-bundle</artifactId>
@@ -35,7 +35,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-html-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-ignite-bundle/nifi-ignite-nar/pom.xml b/nifi-nar-bundles/nifi-ignite-bundle/nifi-ignite-nar/pom.xml
index 5bb1596..8517b9c 100644
--- a/nifi-nar-bundles/nifi-ignite-bundle/nifi-ignite-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-ignite-bundle/nifi-ignite-nar/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-ignite-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-ignite-nar</artifactId>
@@ -33,7 +33,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-ignite-bundle/nifi-ignite-processors/pom.xml b/nifi-nar-bundles/nifi-ignite-bundle/nifi-ignite-processors/pom.xml
index 58e74f7..50e12b1 100644
--- a/nifi-nar-bundles/nifi-ignite-bundle/nifi-ignite-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-ignite-bundle/nifi-ignite-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-ignite-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-ignite-processors</artifactId>
@@ -64,7 +64,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>commons-io</groupId>
@@ -74,7 +74,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-ignite-bundle/pom.xml b/nifi-nar-bundles/nifi-ignite-bundle/pom.xml
index 3ad6e15..0e2cffa 100644
--- a/nifi-nar-bundles/nifi-ignite-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-ignite-bundle/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-ignite-bundle</artifactId>
@@ -44,7 +44,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-ignite-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-nar/pom.xml b/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-nar/pom.xml
index ddf9ba8..32c3cad 100644
--- a/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-nar/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-influxdb-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-influxdb-nar</artifactId>
@@ -33,7 +33,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-processors/pom.xml b/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-processors/pom.xml
index 4ef025e..65306ac 100644
--- a/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-influxdb-bundle/nifi-influxdb-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-influxdb-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-influxdb-processors</artifactId>
@@ -43,12 +43,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-influxdb-bundle/pom.xml b/nifi-nar-bundles/nifi-influxdb-bundle/pom.xml
index 28e53af..6a9f7c2 100644
--- a/nifi-nar-bundles/nifi-influxdb-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-influxdb-bundle/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-influxdb-bundle</artifactId>
@@ -35,7 +35,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-influxdb-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-jetty-bundle/pom.xml b/nifi-nar-bundles/nifi-jetty-bundle/pom.xml
index 85507fd..2907bd4 100644
--- a/nifi-nar-bundles/nifi-jetty-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-jetty-bundle/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-jetty-bundle</artifactId>
     <packaging>nar</packaging>
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service-nar/pom.xml b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service-nar/pom.xml
index 277e382..16e698d 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-jms-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-jms-cf-service-nar</artifactId>
     <packaging>nar</packaging>
@@ -30,13 +30,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-jms-cf-service</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/pom.xml b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/pom.xml
index 5bbf4a8..6c26579 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/pom.xml
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/pom.xml
@@ -12,7 +12,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-jms-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <artifactId>nifi-jms-cf-service</artifactId>
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors-nar/pom.xml b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors-nar/pom.xml
index 0df87d0..e541c5b 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-jms-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-jms-processors-nar</artifactId>
     <packaging>nar</packaging>
@@ -30,13 +30,13 @@
     	<dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-jms-cf-service-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-jms-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/pom.xml b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/pom.xml
index 045235b..d54b5af 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/pom.xml
@@ -12,7 +12,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-jms-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <artifactId>nifi-jms-processors</artifactId>
@@ -30,13 +30,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-jms-cf-service</artifactId>
             <scope>provided</scope>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.springframework</groupId>
@@ -76,7 +76,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-jms-bundle/pom.xml b/nifi-nar-bundles/nifi-jms-bundle/pom.xml
index 65b8020..d2b123f 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-jms-bundle/pom.xml
@@ -17,10 +17,10 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-jms-bundle</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>pom</packaging>
     <description>A bundle of processors that publish to and consume messages from JMS.</description>
     <modules>
diff --git a/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-nar/pom.xml b/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-nar/pom.xml
index eff6a5a..5c79fe0 100644
--- a/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-nar/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-jolt-record-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-jolt-record-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -34,13 +34,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-jolt-record-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-processors/pom.xml b/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-processors/pom.xml
index a5bde6a..38d2c6f 100644
--- a/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-jolt-record-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-jolt-record-processors</artifactId>
@@ -33,12 +33,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -71,25 +71,25 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-serialization-services</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-schema-registry-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-jolt-record-bundle/pom.xml b/nifi-nar-bundles/nifi-jolt-record-bundle/pom.xml
index b75885e..05a9a0f 100644
--- a/nifi-nar-bundles/nifi-jolt-record-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-jolt-record-bundle/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-jolt-record-bundle</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <modules>
@@ -41,17 +41,17 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-jolt-record-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-utils</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-record-serialization-service-api</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>com.bazaarvoice.jolt</groupId>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-nar/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-nar/pom.xml
index 97c4828..71fb6b7 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kafka-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-kafka-0-10-nar</artifactId>
     <packaging>nar</packaging>
@@ -34,7 +34,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/pom.xml
index dc2ed8d..6bccf76 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/pom.xml
@@ -16,7 +16,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kafka-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <artifactId>nifi-kafka-0-10-processors</artifactId>
@@ -37,12 +37,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -83,7 +83,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/pom.xml
index 595d5ac..c9d58a6 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kafka-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-kafka-0-11-nar</artifactId>
     <packaging>nar</packaging>
@@ -34,7 +34,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/pom.xml
index 42d58c4..0294a0d 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/pom.xml
@@ -16,7 +16,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kafka-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <artifactId>nifi-kafka-0-11-processors</artifactId>
@@ -37,12 +37,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -82,7 +82,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-nar/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-nar/pom.xml
index 0e78759..67e61bd 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kafka-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-kafka-0-8-nar</artifactId>
     <packaging>nar</packaging>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/pom.xml
index 3fee025..a6b3f14 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/pom.xml
@@ -16,7 +16,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kafka-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <artifactId>nifi-kafka-0-8-processors</artifactId>
@@ -29,12 +29,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.kafka</groupId>
@@ -65,7 +65,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/pom.xml
index 8d711ce..71552f8 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kafka-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-kafka-0-9-nar</artifactId>
     <packaging>nar</packaging>
@@ -34,7 +34,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/pom.xml
index ad5cbb4..e84082f 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/pom.xml
@@ -16,7 +16,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kafka-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <artifactId>nifi-kafka-0-9-processors</artifactId>
@@ -29,12 +29,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -70,7 +70,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-nar/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-nar/pom.xml
index 31be1ed..cb6510c 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kafka-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-kafka-1-0-nar</artifactId>
     <packaging>nar</packaging>
@@ -34,7 +34,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/pom.xml
index cab0089..74c6474 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/pom.xml
@@ -16,7 +16,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kafka-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <artifactId>nifi-kafka-1-0-processors</artifactId>
@@ -37,17 +37,17 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-sink-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -87,13 +87,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
@@ -105,7 +105,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-path</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-nar/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-nar/pom.xml
index 642ef12..52e1c13 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kafka-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-kafka-2-0-nar</artifactId>
     <packaging>nar</packaging>
@@ -34,7 +34,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/pom.xml
index e66da3a..f22c9a2 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/pom.xml
@@ -16,7 +16,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kafka-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <artifactId>nifi-kafka-2-0-processors</artifactId>
@@ -37,17 +37,17 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-sink-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -88,13 +88,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
@@ -106,7 +106,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-path</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-nar/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-nar/pom.xml
index e2b353a..af874cd 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kafka-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-kafka-2-6-nar</artifactId>
     <packaging>nar</packaging>
@@ -34,7 +34,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/pom.xml
index 1febed4..fc9cd6a 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/pom.xml
@@ -16,7 +16,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kafka-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <artifactId>nifi-kafka-2-6-processors</artifactId>
@@ -37,17 +37,17 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-sink-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -88,13 +88,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
@@ -106,7 +106,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-path</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>compile</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/pom.xml
index abbb6d6..c8d95d2 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-kafka-bundle</artifactId>
     <packaging>pom</packaging>
@@ -53,37 +53,37 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-kafka-0-8-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-kafka-0-10-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-kafka-0-9-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-kafka-0-11-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-kafka-1-0-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
            <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-kafka-2-0-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-kafka-2-6-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <!-- Explicitly force Netty to 3.7.1 due to CVE-2014-0193 -->
diff --git a/nifi-nar-bundles/nifi-kerberos-iaa-providers-bundle/nifi-kerberos-iaa-providers-nar/pom.xml b/nifi-nar-bundles/nifi-kerberos-iaa-providers-bundle/nifi-kerberos-iaa-providers-nar/pom.xml
index 03ffe16..fe95d57 100644
--- a/nifi-nar-bundles/nifi-kerberos-iaa-providers-bundle/nifi-kerberos-iaa-providers-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-kerberos-iaa-providers-bundle/nifi-kerberos-iaa-providers-nar/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kerberos-iaa-providers-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-kerberos-iaa-providers-nar</artifactId>
     <packaging>nar</packaging>
diff --git a/nifi-nar-bundles/nifi-kerberos-iaa-providers-bundle/nifi-kerberos-iaa-providers/pom.xml b/nifi-nar-bundles/nifi-kerberos-iaa-providers-bundle/nifi-kerberos-iaa-providers/pom.xml
index be5932b..ad8fc5d 100644
--- a/nifi-nar-bundles/nifi-kerberos-iaa-providers-bundle/nifi-kerberos-iaa-providers/pom.xml
+++ b/nifi-nar-bundles/nifi-kerberos-iaa-providers-bundle/nifi-kerberos-iaa-providers/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kerberos-iaa-providers-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-kerberos-iaa-providers</artifactId>
     <packaging>jar</packaging>
@@ -37,12 +37,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.springframework.security.kerberos</groupId>
diff --git a/nifi-nar-bundles/nifi-kerberos-iaa-providers-bundle/pom.xml b/nifi-nar-bundles/nifi-kerberos-iaa-providers-bundle/pom.xml
index f857263..c27ed37 100644
--- a/nifi-nar-bundles/nifi-kerberos-iaa-providers-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-kerberos-iaa-providers-bundle/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-kerberos-iaa-providers-bundle</artifactId>
     <packaging>pom</packaging>
@@ -41,7 +41,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-kerberos-iaa-providers</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-nar/pom.xml b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-nar/pom.xml
index a5d35b8..10de8b5 100644
--- a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-nar/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kite-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-kite-nar</artifactId>
@@ -32,7 +32,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hadoop-libraries-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/pom.xml b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/pom.xml
index 48190eb..142b415 100644
--- a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kite-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-kite-processors</artifactId>
@@ -40,7 +40,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
 
         <!-- Kite -->
@@ -202,7 +202,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <!-- must override implicitly dependency on avro to get more recent codec factory options-->
@@ -278,7 +278,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hadoop-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.commons</groupId>
diff --git a/nifi-nar-bundles/nifi-kite-bundle/pom.xml b/nifi-nar-bundles/nifi-kite-bundle/pom.xml
index 7b3646c..fda47fa 100644
--- a/nifi-nar-bundles/nifi-kite-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-kite-bundle/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-kite-bundle</artifactId>
@@ -36,7 +36,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-kite-processors</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <!-- Explicitly force Netty to 3.6.9 due to CVE-2014-0193 -->
diff --git a/nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-controller-service/pom.xml b/nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-controller-service/pom.xml
index d885da9..3fa414d 100644
--- a/nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-controller-service/pom.xml
+++ b/nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-controller-service/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <artifactId>nifi-kudu-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-kudu-controller-service</artifactId>
@@ -65,7 +65,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
@@ -81,7 +81,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-nar/pom.xml b/nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-nar/pom.xml
index 876f2a9..16b7059 100644
--- a/nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-nar/pom.xml
@@ -19,28 +19,28 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kudu-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-kudu-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
 
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-kudu-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-kudu-controller-service</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-hadoop-libraries-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-processors/pom.xml b/nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-processors/pom.xml
index 263ce8e..b655b99 100644
--- a/nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-processors/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-kudu-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-kudu-processors</artifactId>
@@ -58,12 +58,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-kerberos-credentials-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -77,7 +77,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -86,7 +86,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
 
         <dependency>
@@ -98,13 +98,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock-record-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-kudu-bundle/pom.xml b/nifi-nar-bundles/nifi-kudu-bundle/pom.xml
index 894139b..3b9b0fa 100644
--- a/nifi-nar-bundles/nifi-kudu-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-kudu-bundle/pom.xml
@@ -19,12 +19,12 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.nifi</groupId>
     <artifactId>nifi-kudu-bundle</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <modules>
diff --git a/nifi-nar-bundles/nifi-language-translation-bundle/nifi-language-translation-nar/pom.xml b/nifi-nar-bundles/nifi-language-translation-bundle/nifi-language-translation-nar/pom.xml
index 8704fb5..a32d4d9 100644
--- a/nifi-nar-bundles/nifi-language-translation-bundle/nifi-language-translation-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-language-translation-bundle/nifi-language-translation-nar/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-language-translation-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-language-translation-nar</artifactId>
@@ -33,7 +33,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-yandex-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/pom.xml b/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/pom.xml
index a362033..a0d0efe 100644
--- a/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-language-translation-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-yandex-processors</artifactId>
@@ -35,12 +35,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-web-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
 
         <dependency>
@@ -62,7 +62,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-language-translation-bundle/pom.xml b/nifi-nar-bundles/nifi-language-translation-bundle/pom.xml
index 2b78494..4ffe9b0 100644
--- a/nifi-nar-bundles/nifi-language-translation-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-language-translation-bundle/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-language-translation-bundle</artifactId>
diff --git a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers-nar/pom.xml b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers-nar/pom.xml
index b3e3a0e..ceacfa1 100644
--- a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers-nar/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-ldap-iaa-providers-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-ldap-iaa-providers-nar</artifactId>
     <packaging>nar</packaging>
diff --git a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/pom.xml b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/pom.xml
index e4dc4b0..958399c 100644
--- a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/pom.xml
+++ b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-ldap-iaa-providers-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-ldap-iaa-providers</artifactId>
     <packaging>jar</packaging>
@@ -37,12 +37,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -92,7 +92,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-expression-language</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/pom.xml b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/pom.xml
index ab800252..5b84b70 100644
--- a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-ldap-iaa-providers-bundle</artifactId>
     <packaging>pom</packaging>
@@ -41,7 +41,7 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-ldap-iaa-providers</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-nar/pom.xml b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-nar/pom.xml
index 1f7405b..9336ad0 100644
--- a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-nar/pom.xml
@@ -19,23 +19,23 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-lumberjack-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-lumberjack-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
 
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-lumberjack-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/pom.xml b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/pom.xml
index 64a4e7b..54adedb 100644
--- a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-lumberjack-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-lumberjack-processors</artifactId>
@@ -33,28 +33,28 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-socket-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-flowfile-packager</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/pom.xml b/nifi-nar-bundles/nifi-lumberjack-bundle/pom.xml
index 90f67d0..dc85550 100644
--- a/nifi-nar-bundles/nifi-lumberjack-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-lumberjack-bundle/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-lumberjack-bundle</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <modules>
diff --git a/nifi-nar-bundles/nifi-media-bundle/nifi-image-viewer/pom.xml b/nifi-nar-bundles/nifi-media-bundle/nifi-image-viewer/pom.xml
index 9f39e95..f5be6e5 100755
--- a/nifi-nar-bundles/nifi-media-bundle/nifi-image-viewer/pom.xml
+++ b/nifi-nar-bundles/nifi-media-bundle/nifi-image-viewer/pom.xml
@@ -18,7 +18,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-media-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-image-viewer</artifactId>
     <description>NiFi image viewer</description>
diff --git a/nifi-nar-bundles/nifi-media-bundle/nifi-media-nar/pom.xml b/nifi-nar-bundles/nifi-media-bundle/nifi-media-nar/pom.xml
index ccea2f9..4c2b9e0 100644
--- a/nifi-nar-bundles/nifi-media-bundle/nifi-media-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-media-bundle/nifi-media-nar/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-media-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-media-nar</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -34,12 +34,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-media-processors</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-image-viewer</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>war</type>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/pom.xml b/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/pom.xml
index fb3d70a..815a49c 100644
--- a/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/pom.xml
@@ -19,7 +19,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-media-bundle</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-media-processors</artifactId>
@@ -33,12 +33,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-media-bundle/pom.xml b/nifi-nar-bundles/nifi-media-bundle/pom.xml
index 395c0a4..e4856b4 100644
--- a/nifi-nar-bundles/nifi-media-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-media-bundle/pom.xml
@@ -19,11 +19,11 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>nifi-media-bundle</artifactId>
-    <version>1.12.0</version>
+    <version>1.12.1-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <modules>
diff --git a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporter-service-api-nar/pom.xml b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporter-service-api-nar/pom.xml
index 43e3614..f307202 100644
--- a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporter-service-api-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporter-service-api-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <artifactId>nifi-metrics-reporting-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <packaging>nar</packaging>
     <modelVersion>4.0.0</modelVersion>
@@ -27,13 +27,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-metrics-reporter-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporter-service-api/pom.xml b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporter-service-api/pom.xml
index 928b342..bc29fcd 100644
--- a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporter-service-api/pom.xml
+++ b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporter-service-api/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <artifactId>nifi-metrics-reporting-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
diff --git a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-nar/pom.xml b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-nar/pom.xml
index d486ce5..8fad390 100644
--- a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-nar/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <artifactId>nifi-metrics-reporting-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <packaging>nar</packaging>
     <modelVersion>4.0.0</modelVersion>
@@ -28,12 +28,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-metrics-reporting-task</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-metrics-reporter-service-api-nar</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/pom.xml b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/pom.xml
index 2a02cd3..cd750cb 100644
--- a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/pom.xml
+++ b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <artifactId>nifi-metrics-reporting-bundle</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
@@ -27,12 +27,12 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-metrics-reporter-service-api</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
@@ -47,7 +47,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.12.0</version>
+            <version>1.12.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
diff --git a/nifi-nar-bundles/nifi-metrics-reporting-bundle/pom.xml b/nifi-nar-bundles/nifi-metrics-reporting-bundle/pom.xml
index d6df472..93fa861 100644
--- a/nifi-nar-bundles/nifi-metrics-reporting-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-metrics-reporting-bundle/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <artifactId>nifi-nar-bundles</artifactId>
         <groupId>org.apache.nifi</groupId>
-        <version>1.12.0</version>
+        <version>1.12.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
@@ -35,12 +35,12 @@
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-metrics</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-metrics-reporting-task</artifactId>
-                <version>1.12.0</version>
+                <version>1.12.1-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-client-service-api-nar/pom.xml b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-client-service-api-nar/pom.xml
index 426a65d..378b43a 100644
--- a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-client-service-api-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-client-service-api-nar/pom.xml
@@ -19,7 +19,7 @@
... 5904 lines suppressed ...


[nifi] 12/16: NIFI-7740: Add Records Per Transaction and Transactions Per Batch properties to PutHive3Streaming

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.12.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit c10bd4990bfcc5f5fd17c3eefdb03801e7a036a9
Author: Matthew Burgess <ma...@apache.org>
AuthorDate: Mon Aug 24 21:45:00 2020 -0400

    NIFI-7740: Add Records Per Transaction and Transactions Per Batch properties to PutHive3Streaming
    
    NIFI-7740: Incorporated review comments
    
    NIFI-7740: Restore RecordsEOFException superclass to SerializationError
    
    This closes #4489.
    
    Signed-off-by: Peter Turcsanyi <tu...@apache.org>
---
 .../apache/hive/streaming/HiveRecordWriter.java    | 19 +++++--
 .../apache/hive/streaming/RecordsEOFException.java | 28 ++++++++++
 .../nifi/processors/hive/PutHive3Streaming.java    | 63 +++++++++++++++++-----
 .../org/apache/nifi/util/hive/HiveOptions.java     | 12 ++++-
 .../processors/hive/TestPutHive3Streaming.java     |  4 +-
 5 files changed, 106 insertions(+), 20 deletions(-)

diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/HiveRecordWriter.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/HiveRecordWriter.java
index 6edb374..d1b55e9 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/HiveRecordWriter.java
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/HiveRecordWriter.java
@@ -34,14 +34,17 @@ import java.util.Properties;
 
 public class HiveRecordWriter extends AbstractRecordWriter {
 
-    private RecordReader recordReader;
+    private final RecordReader recordReader;
     private NiFiRecordSerDe serde;
-    private ComponentLog log;
+    private final ComponentLog log;
+    private final int recordsPerTransaction;
+    private int currentRecordsWritten;
 
-    public HiveRecordWriter(RecordReader recordReader, ComponentLog log) {
+    public HiveRecordWriter(RecordReader recordReader, ComponentLog log, final int recordsPerTransaction) {
         super(null);
         this.recordReader = recordReader;
         this.log = log;
+        this.recordsPerTransaction = recordsPerTransaction;
     }
 
     @Override
@@ -73,10 +76,16 @@ public class HiveRecordWriter extends AbstractRecordWriter {
     public void write(long writeId, InputStream inputStream) throws StreamingException {
         // The inputStream is already available to the recordReader, so just iterate through the records
         try {
-            Record record;
-            while ((record = recordReader.nextRecord()) != null) {
+            Record record = null;
+            while ((++currentRecordsWritten <= recordsPerTransaction || recordsPerTransaction == 0)
+                    && (record = recordReader.nextRecord()) != null) {
                 write(writeId, record);
             }
+            // Once there are no more records, throw a RecordsEOFException to indicate the input stream is exhausted
+            if (record == null) {
+                throw new RecordsEOFException("End of transaction", new Exception());
+            }
+            currentRecordsWritten = 0;
         } catch (MalformedRecordException | IOException e) {
             throw new StreamingException(e.getLocalizedMessage(), e);
         }
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/RecordsEOFException.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/RecordsEOFException.java
new file mode 100644
index 0000000..41e641b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/RecordsEOFException.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hive.streaming;
+
+/**
+ * This is a "marker class" used by the HiveRecordWriter to indicate there are no more records in the input stream.
+ * It is used by PutHive3Streaming to determine that all records have been written to transaction(s).
+ */
+public class RecordsEOFException extends SerializationError {
+
+    RecordsEOFException(String msg, Exception e) {
+        super(msg, e);
+    }
+}
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java
index 23b873f..0ba6bd2 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java
@@ -24,6 +24,7 @@ import org.apache.hive.streaming.ConnectionError;
 import org.apache.hive.streaming.HiveRecordWriter;
 import org.apache.hive.streaming.HiveStreamingConnection;
 import org.apache.hive.streaming.InvalidTable;
+import org.apache.hive.streaming.RecordsEOFException;
 import org.apache.hive.streaming.SerializationError;
 import org.apache.hive.streaming.StreamingConnection;
 import org.apache.hive.streaming.StreamingException;
@@ -171,6 +172,28 @@ public class PutHive3Streaming extends AbstractProcessor {
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
+    static final PropertyDescriptor RECORDS_PER_TXN = new PropertyDescriptor.Builder()
+            .name("hive3-stream-records-per-transaction")
+            .displayName("Records per Transaction")
+            .description("Number of records to process before committing the transaction. If set to zero (0), all records will be written in a single transaction.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .defaultValue("0")
+            .build();
+
+    static final PropertyDescriptor TXNS_PER_BATCH = new PropertyDescriptor.Builder()
+            .name("hive3-stream-transactions-per-batch")
+            .displayName("Transactions per Batch")
+            .description("A hint to Hive Streaming indicating how many transactions the processor task will need. The product of Records per Transaction (if not zero) "
+                    + "and Transactions per Batch should be larger than the largest expected number of records in the flow file(s), this will ensure any failed "
+                    + "transaction batches cause a full rollback.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .defaultValue("1")
+            .build();
+
     static final PropertyDescriptor CALL_TIMEOUT = new PropertyDescriptor.Builder()
             .name("hive3-stream-call-timeout")
             .displayName("Call Timeout")
@@ -269,6 +292,8 @@ public class PutHive3Streaming extends AbstractProcessor {
         props.add(DB_NAME);
         props.add(TABLE_NAME);
         props.add(STATIC_PARTITION_VALUES);
+        props.add(RECORDS_PER_TXN);
+        props.add(TXNS_PER_BATCH);
         props.add(CALL_TIMEOUT);
         props.add(DISABLE_STREAMING_OPTIMIZATIONS);
         props.add(ROLLBACK_ON_FAILURE);
@@ -355,10 +380,10 @@ public class PutHive3Streaming extends AbstractProcessor {
 
             if (resolvedKeytab != null) {
                 kerberosUserReference.set(new KerberosKeytabUser(resolvedPrincipal, resolvedKeytab));
-                log.info("Hive Security Enabled, logging in as principal {} with keytab {}", new Object[] {resolvedPrincipal, resolvedKeytab});
+                log.info("Hive Security Enabled, logging in as principal {} with keytab {}", new Object[]{resolvedPrincipal, resolvedKeytab});
             } else if (explicitPassword != null) {
                 kerberosUserReference.set(new KerberosPasswordUser(resolvedPrincipal, explicitPassword));
-                log.info("Hive Security Enabled, logging in as principal {} with password", new Object[] {resolvedPrincipal});
+                log.info("Hive Security Enabled, logging in as principal {} with password", new Object[]{resolvedPrincipal});
             } else {
                 throw new ProcessException("Unable to authenticate with Kerberos, no keytab or password was provided");
             }
@@ -409,13 +434,17 @@ public class PutHive3Streaming extends AbstractProcessor {
 
             // Override the Hive Metastore URIs in the config if set by the user
             if (metastoreURIs != null) {
-               hiveConfig.set(MetastoreConf.ConfVars.THRIFT_URIS.getHiveName(), metastoreURIs);
+                hiveConfig.set(MetastoreConf.ConfVars.THRIFT_URIS.getHiveName(), metastoreURIs);
             }
 
+            final int recordsPerTransaction = context.getProperty(RECORDS_PER_TXN).evaluateAttributeExpressions(flowFile).asInteger();
+            final int transactionsPerBatch = context.getProperty(TXNS_PER_BATCH).evaluateAttributeExpressions(flowFile).asInteger();
+
             HiveOptions o = new HiveOptions(metastoreURIs, dbName, tableName)
                     .withHiveConf(hiveConfig)
                     .withCallTimeout(callTimeout)
-                    .withStreamingOptimizations(!disableStreamingOptimizations);
+                    .withStreamingOptimizations(!disableStreamingOptimizations)
+                    .withTransactionBatchSize(transactionsPerBatch);
 
             if (!StringUtils.isEmpty(staticPartitionValuesString)) {
                 List<String> staticPartitionValues = Arrays.stream(staticPartitionValuesString.split(",")).filter(Objects::nonNull).map(String::trim).collect(Collectors.toList());
@@ -444,7 +473,7 @@ public class PutHive3Streaming extends AbstractProcessor {
             try {
                 final RecordReader reader;
 
-                try(final InputStream in = session.read(flowFile)) {
+                try (final InputStream in = session.read(flowFile)) {
                     // if we fail to create the RecordReader then we want to route to failure, so we need to
                     // handle this separately from the other IOExceptions which normally route to retry
                     try {
@@ -453,12 +482,21 @@ public class PutHive3Streaming extends AbstractProcessor {
                         throw new RecordReaderFactoryException("Unable to create RecordReader", e);
                     }
 
-                    hiveStreamingConnection = makeStreamingConnection(options, reader);
+                    hiveStreamingConnection = makeStreamingConnection(options, reader, recordsPerTransaction);
 
                     // Write records to Hive streaming, then commit and close
-                    hiveStreamingConnection.beginTransaction();
-                    hiveStreamingConnection.write(in);
-                    hiveStreamingConnection.commitTransaction();
+                    boolean exitLoop = false;
+                    while (!exitLoop) {
+                        hiveStreamingConnection.beginTransaction();
+                        // The HiveRecordWriter keeps track of records per transaction and will complete writing for the transaction
+                        // once the limit has been reached. It is then reset for the next iteration of the loop.
+                        try {
+                            hiveStreamingConnection.write(in);
+                        } catch (RecordsEOFException reofe) {
+                            exitLoop = true;
+                        }
+                        hiveStreamingConnection.commitTransaction();
+                    }
                     in.close();
 
                     Map<String, String> updateAttributes = new HashMap<>();
@@ -560,13 +598,14 @@ public class PutHive3Streaming extends AbstractProcessor {
         });
     }
 
-    StreamingConnection makeStreamingConnection(HiveOptions options, RecordReader reader) throws StreamingException {
+    StreamingConnection makeStreamingConnection(HiveOptions options, RecordReader reader, int recordsPerTransaction) throws StreamingException {
         return HiveStreamingConnection.newBuilder()
                 .withDatabase(options.getDatabaseName())
                 .withTable(options.getTableName())
                 .withStaticPartitionValues(options.getStaticPartitionValues())
                 .withHiveConf(options.getHiveConf())
-                .withRecordWriter(new HiveRecordWriter(reader, getLogger()))
+                .withRecordWriter(new HiveRecordWriter(reader, getLogger(), recordsPerTransaction))
+                .withTransactionBatchSize(options.getTransactionBatchSize())
                 .withAgentInfo("NiFi " + this.getClass().getSimpleName() + " [" + this.getIdentifier()
                         + "] thread " + Thread.currentThread().getId() + "[" + Thread.currentThread().getName() + "]")
                 .connect();
@@ -642,7 +681,7 @@ public class PutHive3Streaming extends AbstractProcessor {
             KerberosUser kerberosUser = kerberosUserReference.get();
             getLogger().debug("kerberosUser is " + kerberosUser);
             try {
-                getLogger().debug("checking TGT on kerberosUser [{}]", new Object[] {kerberosUser});
+                getLogger().debug("checking TGT on kerberosUser [{}]", new Object[]{kerberosUser});
                 kerberosUser.checkTGTAndRelogin();
             } catch (LoginException e) {
                 throw new ProcessException("Unable to relogin with kerberos credentials for " + kerberosUser.getPrincipal(), e);
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveOptions.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveOptions.java
index 82f6856..7efa106 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveOptions.java
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveOptions.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -36,6 +36,7 @@ public class HiveOptions implements Serializable {
     protected String kerberosKeytab;
     protected HiveConf hiveConf;
     protected boolean streamingOptimizations = true;
+    protected int transactionBatchSize = 1;
 
     public HiveOptions(String metaStoreURI, String databaseName, String tableName) {
         this.metaStoreURI = metaStoreURI;
@@ -73,6 +74,11 @@ public class HiveOptions implements Serializable {
         return this;
     }
 
+    public HiveOptions withTransactionBatchSize(int transactionBatchSize) {
+        this.transactionBatchSize = transactionBatchSize;
+        return this;
+    }
+
     public String getMetaStoreURI() {
         return metaStoreURI;
     }
@@ -108,4 +114,8 @@ public class HiveOptions implements Serializable {
     public boolean getStreamingOptimizations() {
         return streamingOptimizations;
     }
+
+    public int getTransactionBatchSize() {
+        return transactionBatchSize;
+    }
 }
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java
index 2b6487e..05e44fb 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java
@@ -1142,7 +1142,7 @@ public class TestPutHive3Streaming {
         }
 
         @Override
-        StreamingConnection makeStreamingConnection(HiveOptions options, RecordReader reader) throws StreamingException {
+        StreamingConnection makeStreamingConnection(HiveOptions options, RecordReader reader, int recordsPerTransaction) throws StreamingException {
 
             // Test here to ensure the 'hive.metastore.uris' property matches the options.getMetastoreUri() value (if it is set)
             String userDefinedMetastoreURI = options.getMetaStoreURI();
@@ -1154,7 +1154,7 @@ public class TestPutHive3Streaming {
                 throw new StubConnectionError("Unit Test - Connection Error");
             }
 
-            HiveRecordWriter hiveRecordWriter = new HiveRecordWriter(reader, getLogger());
+            HiveRecordWriter hiveRecordWriter = new HiveRecordWriter(reader, getLogger(), 0);
             if (generatePermissionsFailure) {
                 throw new StreamingException("Permission denied");
             }


[nifi] 08/16: NIFI-6767 Persist registry URL after service restart.

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.12.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 3ef212ea985a04494ecdc319416fa55cdb5601fd
Author: Kristjan Antunovic <kr...@gmail.com>
AuthorDate: Wed Jan 22 16:01:49 2020 +0100

    NIFI-6767 Persist registry URL after service restart.
    
    This closes #4006.
    
    Signed-off-by: Bryan Bende <bb...@apache.org>
---
 .../org/apache/nifi/registry/flow/StandardFlowRegistryClient.java | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/StandardFlowRegistryClient.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/StandardFlowRegistryClient.java
index 265eb2c..ed4feb9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/StandardFlowRegistryClient.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/StandardFlowRegistryClient.java
@@ -28,6 +28,8 @@ import org.apache.nifi.security.util.TlsConfiguration;
 import org.apache.nifi.security.util.TlsException;
 import org.apache.nifi.util.NiFiProperties;
 
+import org.apache.http.client.utils.URIBuilder;
+
 public class StandardFlowRegistryClient implements FlowRegistryClient {
     private NiFiProperties nifiProperties;
     private ConcurrentMap<String, FlowRegistry> registryById = new ConcurrentHashMap<>();
@@ -61,7 +63,8 @@ public class StandardFlowRegistryClient implements FlowRegistryClient {
     public FlowRegistry addFlowRegistry(final String registryId, final String registryName, final String registryUrl, final String description) {
         final URI uri;
         try {
-            uri = new URI(registryUrl);
+            // Handles case where the URI entered has a trailing slash, or includes the trailing /nifi-registry-api
+            uri = new URIBuilder(registryUrl).setPath("").removeQuery().build();
         } catch (URISyntaxException e) {
             throw new IllegalArgumentException("The given Registry URL is not valid: " + registryUrl);
         }
@@ -71,8 +74,7 @@ public class StandardFlowRegistryClient implements FlowRegistryClient {
             throw new IllegalArgumentException("The given Registry URL is not valid: " + registryUrl);
         }
 
-        // Handles case where the URI entered has a trailing slash, or includes the trailing /nifi-registry-api
-        final String registryBaseUrl = uri.getScheme() + "://" + uri.getHost() + ":" + uri.getPort();
+        final String registryBaseUrl = uri.toString();
 
         final FlowRegistry registry;
         if (uriScheme.equalsIgnoreCase("http") || uriScheme.equalsIgnoreCase("https")) {


[nifi] 06/16: NIFI-7714: QueryCassandra loses precision when converting timestamps to JSON

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.12.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 46923810641514d97f55781985959413bee6f28a
Author: Denes Arvay <de...@apache.org>
AuthorDate: Fri Aug 7 14:27:40 2020 +0200

    NIFI-7714: QueryCassandra loses precision when converting timestamps to JSON
    
    Updated the patch based on @tpalfy's review
    Updated the patch based on @mattyb149's review
    Rename DATE_FORMAT_PATTERN to JSON_TIMESTAMP_FORMAT_PATTERN
    Changed convertToJsonStream method's visibility to package private.
    Removed json prefix from timestamp-format-pattern property to make it more generic
    
    This closes #4463.
    
    Signed-off-by: Peter Turcsanyi <tu...@apache.org>
---
 .../nifi/processors/cassandra/QueryCassandra.java  | 58 ++++++++++++++++++----
 .../cassandra/CassandraQueryTestUtil.java          | 32 ++++++++++++
 .../processors/cassandra/QueryCassandraTest.java   | 48 ++++++++++++++++++
 3 files changed, 129 insertions(+), 9 deletions(-)

diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/QueryCassandra.java b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/QueryCassandra.java
index ebad736..6212082 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/QueryCassandra.java
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/QueryCassandra.java
@@ -25,6 +25,7 @@ import com.datastax.driver.core.Session;
 import com.datastax.driver.core.exceptions.NoHostAvailableException;
 import com.datastax.driver.core.exceptions.QueryExecutionException;
 import com.datastax.driver.core.exceptions.QueryValidationException;
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.avro.Schema;
 import org.apache.avro.SchemaBuilder;
 import org.apache.avro.file.DataFileWriter;
@@ -39,6 +40,7 @@ import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.lifecycle.OnShutdown;
 import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
@@ -68,6 +70,7 @@ import java.util.Date;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.TimeZone;
 import java.util.concurrent.ExecutionException;
@@ -130,6 +133,24 @@ public class QueryCassandra extends AbstractCassandraProcessor {
             .defaultValue(AVRO_FORMAT)
             .build();
 
+    public static final PropertyDescriptor TIMESTAMP_FORMAT_PATTERN = new PropertyDescriptor.Builder()
+            .name("timestamp-format-pattern")
+            .displayName("Timestamp Format Pattern for JSON output")
+            .description("Pattern to use when converting timestamp fields to JSON. Note: the formatted timestamp will be in UTC timezone.")
+            .required(true)
+            .defaultValue("yyyy-MM-dd HH:mm:ssZ")
+            .addValidator((subject, input, context) -> {
+                final ValidationResult.Builder vrb = new ValidationResult.Builder().subject(subject).input(input);
+                try {
+                    new SimpleDateFormat(input).format(new Date());
+                    vrb.valid(true).explanation("Valid date format pattern");
+                } catch (Exception ex) {
+                    vrb.valid(false).explanation("the pattern is invalid: " + ex.getMessage());
+                }
+                return vrb.build();
+            })
+            .build();
+
     private final static List<PropertyDescriptor> propertyDescriptors;
 
     private final static Set<Relationship> relationships;
@@ -145,6 +166,7 @@ public class QueryCassandra extends AbstractCassandraProcessor {
         _propertyDescriptors.add(QUERY_TIMEOUT);
         _propertyDescriptors.add(FETCH_SIZE);
         _propertyDescriptors.add(OUTPUT_FORMAT);
+        _propertyDescriptors.add(TIMESTAMP_FORMAT_PATTERN);
         propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
 
         Set<Relationship> _relationships = new HashSet<>();
@@ -220,14 +242,14 @@ public class QueryCassandra extends AbstractCassandraProcessor {
                             if (AVRO_FORMAT.equals(outputFormat)) {
                                 nrOfRows.set(convertToAvroStream(resultSet, out, queryTimeout, TimeUnit.MILLISECONDS));
                             } else if (JSON_FORMAT.equals(outputFormat)) {
-                                nrOfRows.set(convertToJsonStream(resultSet, out, charset, queryTimeout, TimeUnit.MILLISECONDS));
+                                nrOfRows.set(convertToJsonStream(Optional.of(context), resultSet, out, charset, queryTimeout, TimeUnit.MILLISECONDS));
                             }
                         } else {
                             resultSet = queryFuture.getUninterruptibly();
                             if (AVRO_FORMAT.equals(outputFormat)) {
                                 nrOfRows.set(convertToAvroStream(resultSet, out, 0, null));
                             } else if (JSON_FORMAT.equals(outputFormat)) {
-                                nrOfRows.set(convertToJsonStream(resultSet, out, charset, 0, null));
+                                nrOfRows.set(convertToJsonStream(Optional.of(context), resultSet, out, charset, 0, null));
                             }
                         }
 
@@ -381,6 +403,13 @@ public class QueryCassandra extends AbstractCassandraProcessor {
      */
     public static long convertToJsonStream(final ResultSet rs, final OutputStream outStream,
                                            Charset charset, long timeout, TimeUnit timeUnit)
+        throws IOException, InterruptedException, TimeoutException, ExecutionException {
+        return convertToJsonStream(Optional.empty(), rs, outStream, charset, timeout, timeUnit);
+    }
+
+    @VisibleForTesting
+    static long convertToJsonStream(final Optional<ProcessContext> context, final ResultSet rs, final OutputStream outStream,
+                                           Charset charset, long timeout, TimeUnit timeUnit)
             throws IOException, InterruptedException, TimeoutException, ExecutionException {
 
         try {
@@ -425,7 +454,7 @@ public class QueryCassandra extends AbstractCassandraProcessor {
                                         if (!first) {
                                             sb.append(",");
                                         }
-                                        sb.append(getJsonElement(element));
+                                        sb.append(getJsonElement(context, element));
                                         first = false;
                                     }
                                     sb.append("]");
@@ -441,15 +470,15 @@ public class QueryCassandra extends AbstractCassandraProcessor {
                                         if (!first) {
                                             sb.append(",");
                                         }
-                                        sb.append(getJsonElement(mapKey));
+                                        sb.append(getJsonElement(context, mapKey));
                                         sb.append(":");
-                                        sb.append(getJsonElement(mapValue));
+                                        sb.append(getJsonElement(context, mapValue));
                                         first = false;
                                     }
                                     sb.append("}");
                                     valueString = sb.toString();
                                 } else {
-                                    valueString = getJsonElement(value);
+                                    valueString = getJsonElement(context, value);
                                 }
                                 outStream.write(("\"" + colName + "\":"
                                         + valueString + "").getBytes(charset));
@@ -467,12 +496,14 @@ public class QueryCassandra extends AbstractCassandraProcessor {
     }
 
     protected static String getJsonElement(Object value) {
+        return getJsonElement(Optional.empty(), value);
+    }
+
+    protected static String getJsonElement(final Optional<ProcessContext> context, Object value) {
         if (value instanceof Number) {
             return value.toString();
         } else if (value instanceof Date) {
-            SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ssZ");
-            dateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
-            return "\"" + dateFormat.format((Date) value) + "\"";
+            return "\"" + getFormattedDate(context, (Date) value) + "\"";
         } else if (value instanceof String) {
             return "\"" + StringEscapeUtils.escapeJson((String) value) + "\"";
         } else {
@@ -480,6 +511,15 @@ public class QueryCassandra extends AbstractCassandraProcessor {
         }
     }
 
+    private static String getFormattedDate(final Optional<ProcessContext> context, Date value) {
+        final String dateFormatPattern = context
+                .map(_context -> _context.getProperty(TIMESTAMP_FORMAT_PATTERN).getValue())
+                .orElse(TIMESTAMP_FORMAT_PATTERN.getDefaultValue());
+        SimpleDateFormat dateFormat = new SimpleDateFormat(dateFormatPattern);
+        dateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
+        return dateFormat.format(value);
+    }
+
     /**
      * Creates an Avro schema from the given result set. The metadata (column definitions, data types, etc.) is used
      * to determine a schema for Avro.
diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/test/java/org/apache/nifi/processors/cassandra/CassandraQueryTestUtil.java b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/test/java/org/apache/nifi/processors/cassandra/CassandraQueryTestUtil.java
index ab85e9f..d5e5a08 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/test/java/org/apache/nifi/processors/cassandra/CassandraQueryTestUtil.java
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/test/java/org/apache/nifi/processors/cassandra/CassandraQueryTestUtil.java
@@ -27,8 +27,10 @@ import org.mockito.stubbing.Answer;
 
 import java.text.SimpleDateFormat;
 import java.util.Arrays;
+import java.util.Calendar;
 import java.util.Collections;
 import java.util.Date;
+import java.util.GregorianCalendar;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -45,6 +47,15 @@ import static org.mockito.Mockito.when;
  * Utility methods for Cassandra processors' unit tests
  */
 public class CassandraQueryTestUtil {
+
+    static final Date TEST_DATE;
+    static {
+        Calendar c = GregorianCalendar.getInstance(TimeZone.getTimeZone("PST"));
+        c.set(2020, Calendar.JANUARY, 1, 10, 10, 10);
+        c.set(Calendar.MILLISECOND, 10);
+        TEST_DATE = c.getTime();
+    }
+
     public static ResultSet createMockResultSet() throws Exception {
         ResultSet resultSet = mock(ResultSet.class);
         ColumnDefinitions columnDefinitions = mock(ColumnDefinitions.class);
@@ -140,6 +151,27 @@ public class CassandraQueryTestUtil {
         return resultSet;
     }
 
+    public static ResultSet createMockDateResultSet() throws Exception {
+        ResultSet resultSet = mock(ResultSet.class);
+        ColumnDefinitions columnDefinitions = mock(ColumnDefinitions.class);
+
+        when(columnDefinitions.size()).thenReturn(1);
+        when(columnDefinitions.getName(anyInt())).thenReturn("date");
+        when(columnDefinitions.getTable(0)).thenReturn("users");
+        when(columnDefinitions.getType(anyInt())).thenReturn(DataType.timestamp());
+
+        Row row = mock(Row.class);
+        when(row.getTimestamp(0)).thenReturn(TEST_DATE);
+        List<Row> rows = Collections.singletonList(row);
+
+        when(resultSet.iterator()).thenReturn(rows.iterator());
+        when(resultSet.all()).thenReturn(rows);
+        when(resultSet.getAvailableWithoutFetching()).thenReturn(rows.size());
+        when(resultSet.isFullyFetched()).thenReturn(false).thenReturn(true);
+        when(resultSet.getColumnDefinitions()).thenReturn(columnDefinitions);
+        return resultSet;
+    }
+
     public static Row createRow(String user_id, String first_name, String last_name, Set<String> emails,
                                 List<String> top_places, Map<Date, String> todo, boolean registered,
                                 float scale, double metric) {
diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/test/java/org/apache/nifi/processors/cassandra/QueryCassandraTest.java b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/test/java/org/apache/nifi/processors/cassandra/QueryCassandraTest.java
index dd6301f..330b776 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/test/java/org/apache/nifi/processors/cassandra/QueryCassandraTest.java
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/test/java/org/apache/nifi/processors/cassandra/QueryCassandraTest.java
@@ -39,16 +39,23 @@ import com.datastax.driver.core.exceptions.ReadTimeoutException;
 import java.io.ByteArrayOutputStream;
 import java.net.InetSocketAddress;
 import java.nio.charset.StandardCharsets;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.TimeZone;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import javax.net.ssl.SSLContext;
 import org.apache.avro.Schema;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.MockProcessContext;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
+import org.codehaus.jackson.map.ObjectMapper;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -75,6 +82,11 @@ public class QueryCassandraTest {
         testRunner.assertNotValid();
         testRunner.setProperty(AbstractCassandraProcessor.USERNAME, "username");
         testRunner.assertValid();
+
+        testRunner.setProperty(QueryCassandra.TIMESTAMP_FORMAT_PATTERN, "invalid format");
+        testRunner.assertNotValid();
+        testRunner.setProperty(QueryCassandra.TIMESTAMP_FORMAT_PATTERN, "yyyy-MM-dd HH:mm:ss.SSSZ");
+        testRunner.assertValid();
     }
 
     @Test
@@ -368,6 +380,42 @@ public class QueryCassandraTest {
         assertEquals(2, numberOfRows);
     }
 
+    @Test
+    public void testDefaultDateFormatInConvertToJSONStream() throws Exception {
+        ResultSet rs = CassandraQueryTestUtil.createMockDateResultSet();
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+        DateFormat df = new SimpleDateFormat(QueryCassandra.TIMESTAMP_FORMAT_PATTERN.getDefaultValue());
+        df.setTimeZone(TimeZone.getTimeZone("UTC"));
+
+        long numberOfRows = QueryCassandra.convertToJsonStream(Optional.of(testRunner.getProcessContext()), rs, baos,
+            StandardCharsets.UTF_8, 0, null);
+        assertEquals(1, numberOfRows);
+
+        Map<String, List<Map<String, String>>> map = new ObjectMapper().readValue(baos.toByteArray(), HashMap.class);
+        String date = map.get("results").get(0).get("date");
+        assertEquals(df.format(CassandraQueryTestUtil.TEST_DATE), date);
+    }
+
+    @Test
+    public void testCustomDateFormatInConvertToJSONStream() throws Exception {
+        MockProcessContext context = (MockProcessContext) testRunner.getProcessContext();
+        ResultSet rs = CassandraQueryTestUtil.createMockDateResultSet();
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+        final String customDateFormat = "yyyy-MM-dd HH:mm:ss.SSSZ";
+        context.setProperty(QueryCassandra.TIMESTAMP_FORMAT_PATTERN, customDateFormat);
+        DateFormat df = new SimpleDateFormat(customDateFormat);
+        df.setTimeZone(TimeZone.getTimeZone("UTC"));
+
+        long numberOfRows = QueryCassandra.convertToJsonStream(Optional.of(context), rs, baos, StandardCharsets.UTF_8, 0, null);
+        assertEquals(1, numberOfRows);
+
+        Map<String, List<Map<String, String>>> map = new ObjectMapper().readValue(baos.toByteArray(), HashMap.class);
+        String date = map.get("results").get(0).get("date");
+        assertEquals(df.format(CassandraQueryTestUtil.TEST_DATE), date);
+    }
+
     private void setUpStandardProcessorConfig() {
         testRunner.setProperty(AbstractCassandraProcessor.CONSISTENCY_LEVEL, "ONE");
         testRunner.setProperty(AbstractCassandraProcessor.CONTACT_POINTS, "localhost:9042");


[nifi] 16/16: NIFI-7167 Fixing resource leaks in IdentifyMimeType

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.12.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 353a91beb8de5a903c6b01f43bbc0d6e1ca651ae
Author: Cameron E. Tidd <ct...@TC-PC10.TC.technicallycreative.com>
AuthorDate: Thu Aug 20 20:56:00 2020 -0400

    NIFI-7167 Fixing resource leaks in IdentifyMimeType
    
    The TikaInputStream and FileInputStream instances utilized in IdentifyMimeType are now explicitly closed. The FileInputStream is additionally wrapped by a BufferedInputStream.
---
 .../org/apache/nifi/processors/standard/IdentifyMimeType.java    | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java
index d6ebd39..90b4c41 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java
@@ -177,8 +177,9 @@ public class IdentifyMimeType extends AbstractProcessor {
             }
 
         } else {
-            try {
-                this.detector = MimeTypesFactory.create(new FileInputStream(configFile));
+            try (final FileInputStream fis = new FileInputStream(configFile);
+                 final InputStream bis = new BufferedInputStream(fis)) {
+                this.detector = MimeTypesFactory.create(bis);
                 this.mimeTypes = (MimeTypes)this.detector;
             } catch (Exception e) {
                 context.yield();
@@ -212,8 +213,8 @@ public class IdentifyMimeType extends AbstractProcessor {
         session.read(flowFile, new InputStreamCallback() {
             @Override
             public void process(final InputStream stream) throws IOException {
-                try (final InputStream in = new BufferedInputStream(stream)) {
-                    TikaInputStream tikaStream = TikaInputStream.get(in);
+                try (final InputStream in = new BufferedInputStream(stream);
+                     final TikaInputStream tikaStream = TikaInputStream.get(in)) {
                     Metadata metadata = new Metadata();
 
                     if (filename != null && context.getProperty(USE_FILENAME_IN_DETECTION).asBoolean()) {


[nifi] 14/16: NIFI-7767 - Fixed issue with tls-toolkit not adding SANs to generated certificates. Added tests. NIFI-7767 - Fixed up TlsCertificateAuthorityTest to include SAN in tests.

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.12.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit e2bf1f28c837e498385ac9ef2ce5ac51ad935c2d
Author: Nathan Gough <th...@gmail.com>
AuthorDate: Fri Aug 28 16:33:51 2020 -0400

    NIFI-7767 - Fixed issue with tls-toolkit not adding SANs to generated certificates. Added tests.
    NIFI-7767 - Fixed up TlsCertificateAuthorityTest to include SAN in tests.
---
 .../nifi/security/util/CertificateUtils.java       |  3 +-
 .../nifi/security/util/CertificateUtilsTest.groovy | 51 +++++++++++++
 .../tls/service/TlsCertificateAuthorityTest.java   | 27 +++++++
 .../TlsCertificateAuthorityServiceHandlerTest.java | 88 ++++++++++++++++------
 4 files changed, 145 insertions(+), 24 deletions(-)

diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/CertificateUtils.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/CertificateUtils.java
index ee0c33e..a93c518 100644
--- a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/CertificateUtils.java
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/CertificateUtils.java
@@ -50,6 +50,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.bouncycastle.asn1.ASN1Encodable;
 import org.bouncycastle.asn1.ASN1ObjectIdentifier;
 import org.bouncycastle.asn1.ASN1Set;
+import org.bouncycastle.asn1.DLSequence;
 import org.bouncycastle.asn1.DERSequence;
 import org.bouncycastle.asn1.pkcs.Attribute;
 import org.bouncycastle.asn1.pkcs.PKCSObjectIdentifiers;
@@ -598,7 +599,7 @@ public final class CertificateUtils {
                 ASN1Encodable extension = attValue.getObjectAt(0);
                 if (extension instanceof Extensions) {
                     return (Extensions) extension;
-                } else if (extension instanceof DERSequence) {
+                } else if (extension instanceof DERSequence || extension instanceof DLSequence) {
                     return Extensions.getInstance(extension);
                 }
             }
diff --git a/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/CertificateUtilsTest.groovy b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/CertificateUtilsTest.groovy
index d20cfeb..a1044ca 100644
--- a/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/CertificateUtilsTest.groovy
+++ b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/CertificateUtilsTest.groovy
@@ -16,12 +16,20 @@
  */
 package org.apache.nifi.security.util
 
+import org.bouncycastle.asn1.pkcs.PKCSObjectIdentifiers
+import org.bouncycastle.asn1.x500.X500Name
+import org.bouncycastle.asn1.x500.style.BCStyle
+import org.bouncycastle.asn1.x500.style.IETFUtils
 import org.bouncycastle.asn1.x509.Extension
 import org.bouncycastle.asn1.x509.Extensions
 import org.bouncycastle.asn1.x509.ExtensionsGenerator
 import org.bouncycastle.asn1.x509.GeneralName
 import org.bouncycastle.asn1.x509.GeneralNames
 import org.bouncycastle.operator.OperatorCreationException
+import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder
+import org.bouncycastle.pkcs.jcajce.JcaPKCS10CertificationRequest
+import org.bouncycastle.pkcs.jcajce.JcaPKCS10CertificationRequestBuilder
+import org.bouncycastle.util.IPAddress
 import org.junit.After
 import org.junit.Before
 import org.junit.BeforeClass
@@ -68,6 +76,7 @@ class CertificateUtilsTest extends GroovyTestCase {
 
     private static final String SUBJECT_DN = "CN=NiFi Test Server,OU=Security,O=Apache,ST=CA,C=US"
     private static final String ISSUER_DN = "CN=NiFi Test CA,OU=Security,O=Apache,ST=CA,C=US"
+    private static final List<String> SUBJECT_ALT_NAMES = ["127.0.0.1", "nifi.nifi.apache.org"]
 
     @BeforeClass
     static void setUpOnce() {
@@ -655,4 +664,46 @@ class CertificateUtilsTest extends GroovyTestCase {
             assert tlsVersion == "TLSv1.3"
         }
     }
+
+    @Test
+    void testGetExtensionsFromCSR() {
+        // Arrange
+        KeyPairGenerator generator = KeyPairGenerator.getInstance("RSA")
+        KeyPair keyPair = generator.generateKeyPair()
+        Extensions sanExtensions = createDomainAlternativeNamesExtensions(SUBJECT_ALT_NAMES, SUBJECT_DN)
+
+        JcaPKCS10CertificationRequestBuilder jcaPKCS10CertificationRequestBuilder = new JcaPKCS10CertificationRequestBuilder(new X500Name(SUBJECT_DN), keyPair.getPublic())
+        jcaPKCS10CertificationRequestBuilder.addAttribute(PKCSObjectIdentifiers.pkcs_9_at_extensionRequest, sanExtensions)
+        JcaContentSignerBuilder jcaContentSignerBuilder = new JcaContentSignerBuilder("SHA256WITHRSA")
+        JcaPKCS10CertificationRequest jcaPKCS10CertificationRequest = new JcaPKCS10CertificationRequest(jcaPKCS10CertificationRequestBuilder.build(jcaContentSignerBuilder.build(keyPair.getPrivate())))
+
+        // Act
+        Extensions extensions = CertificateUtils.getExtensionsFromCSR(jcaPKCS10CertificationRequest)
+
+        // Assert
+        assert(extensions.equivalent(sanExtensions))
+    }
+
+    // Using this directly from tls-toolkit results in a dependency loop, so it's added here for testing purposes.
+    private static Extensions createDomainAlternativeNamesExtensions(List<String> domainAlternativeNames, String requestedDn) throws IOException {
+        List<GeneralName> namesList = new ArrayList<>()
+
+        try {
+            final String cn = IETFUtils.valueToString(new X500Name(requestedDn).getRDNs(BCStyle.CN)[0].getFirst().getValue())
+            namesList.add(new GeneralName(GeneralName.dNSName, cn))
+        } catch (Exception e) {
+            throw new IOException("Failed to extract CN from request DN: " + requestedDn, e)
+        }
+
+        if (domainAlternativeNames != null) {
+            for (String alternativeName : domainAlternativeNames) {
+                namesList.add(new GeneralName(IPAddress.isValid(alternativeName) ? GeneralName.iPAddress : GeneralName.dNSName, alternativeName))
+            }
+        }
+
+        GeneralNames subjectAltNames = new GeneralNames(namesList.toArray([] as GeneralName[]))
+        ExtensionsGenerator extGen = new ExtensionsGenerator()
+        extGen.addExtension(Extension.subjectAlternativeName, false, subjectAltNames)
+        return extGen.generate()
+    }
 }
diff --git a/nifi-toolkit/nifi-toolkit-tls/src/test/java/org/apache/nifi/toolkit/tls/service/TlsCertificateAuthorityTest.java b/nifi-toolkit/nifi-toolkit-tls/src/test/java/org/apache/nifi/toolkit/tls/service/TlsCertificateAuthorityTest.java
index 7137dfe..afefee3 100644
--- a/nifi-toolkit/nifi-toolkit-tls/src/test/java/org/apache/nifi/toolkit/tls/service/TlsCertificateAuthorityTest.java
+++ b/nifi-toolkit/nifi-toolkit-tls/src/test/java/org/apache/nifi/toolkit/tls/service/TlsCertificateAuthorityTest.java
@@ -51,6 +51,11 @@ import java.security.SignatureException;
 import java.security.UnrecoverableEntryException;
 import java.security.cert.Certificate;
 import java.security.cert.CertificateException;
+import java.security.cert.CertificateParsingException;
+import java.security.cert.X509Certificate;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -73,6 +78,7 @@ public class TlsCertificateAuthorityTest {
     private ByteArrayOutputStream clientTrustStoreOutputStream;
     private ByteArrayOutputStream serverConfigFileOutputStream;
     private ByteArrayOutputStream clientConfigFileOutputStream;
+    private String subjectAlternativeName;
 
     @Before
     public void setup() throws FileNotFoundException {
@@ -87,6 +93,7 @@ public class TlsCertificateAuthorityTest {
         clientTrustStoreOutputStream = new ByteArrayOutputStream();
         serverConfigFileOutputStream = new ByteArrayOutputStream();
         clientConfigFileOutputStream = new ByteArrayOutputStream();
+        subjectAlternativeName = "nifi.apache.org";
 
         String myTestTokenUseSomethingStronger = "myTestTokenUseSomethingStronger";
         int port = availablePort();
@@ -106,6 +113,7 @@ public class TlsCertificateAuthorityTest {
         clientConfig.setKeyStore(clientKeyStore);
         clientConfig.setTrustStore(clientTrustStore);
         clientConfig.setToken(myTestTokenUseSomethingStronger);
+        clientConfig.setDomainAlternativeNames(Arrays.asList(subjectAlternativeName));
         clientConfig.setPort(port);
         clientConfig.setKeySize(2048);
         clientConfig.initDefaults();
@@ -240,6 +248,9 @@ public class TlsCertificateAuthorityTest {
         certificateChain[0].verify(caCertificate.getPublicKey());
         assertPrivateAndPublicKeyMatch(clientPrivateKeyEntry.getPrivateKey(), certificateChain[0].getPublicKey());
 
+        // Does the certificate contain the SAN we defined in the client config?
+        assert(isSANPresent(certificateChain[0]));
+
         KeyStore clientTrustStore = KeyStoreUtils.getTrustStore(KeystoreType.JKS.toString());
         clientTrustStore.load(new ByteArrayInputStream(clientTrustStoreOutputStream.toByteArray()), clientConfig.getTrustStorePassword().toCharArray());
         assertEquals(caCertificate, clientTrustStore.getCertificate(TlsToolkitStandalone.NIFI_CERT));
@@ -257,6 +268,22 @@ public class TlsCertificateAuthorityTest {
         verify.verify(signature.sign());
     }
 
+    private boolean isSANPresent(Certificate cert) {
+        Iterator<List<?>> iterator = null;
+        try {
+            iterator = ((X509Certificate) cert).getSubjectAlternativeNames().iterator();
+        } catch (CertificateParsingException e) {
+            e.printStackTrace();
+        }
+        boolean containsSAN = false;
+        while(iterator.hasNext()) {
+            if(iterator.next().contains(subjectAlternativeName)) {
+                containsSAN = true;
+            }
+        }
+        return containsSAN;
+    }
+
     /**
      * Will determine the available port used by ca server
      */
diff --git a/nifi-toolkit/nifi-toolkit-tls/src/test/java/org/apache/nifi/toolkit/tls/service/server/TlsCertificateAuthorityServiceHandlerTest.java b/nifi-toolkit/nifi-toolkit-tls/src/test/java/org/apache/nifi/toolkit/tls/service/server/TlsCertificateAuthorityServiceHandlerTest.java
index eb0dbc3..54e49ca 100644
--- a/nifi-toolkit/nifi-toolkit-tls/src/test/java/org/apache/nifi/toolkit/tls/service/server/TlsCertificateAuthorityServiceHandlerTest.java
+++ b/nifi-toolkit/nifi-toolkit-tls/src/test/java/org/apache/nifi/toolkit/tls/service/server/TlsCertificateAuthorityServiceHandlerTest.java
@@ -17,15 +17,48 @@
 
 package org.apache.nifi.toolkit.tls.service.server;
 
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
 import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringReader;
+import java.io.StringWriter;
+import java.nio.charset.StandardCharsets;
+import java.security.GeneralSecurityException;
+import java.security.InvalidKeyException;
+import java.security.KeyPair;
+import java.security.KeyPairGenerator;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+import java.security.cert.X509Certificate;
+import java.util.Arrays;
+import java.util.List;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
 import org.apache.nifi.security.util.CertificateUtils;
 import org.apache.nifi.toolkit.tls.configuration.TlsConfig;
 import org.apache.nifi.toolkit.tls.service.dto.TlsCertificateAuthorityRequest;
 import org.apache.nifi.toolkit.tls.service.dto.TlsCertificateAuthorityResponse;
 import org.apache.nifi.toolkit.tls.util.TlsHelper;
+import org.bouncycastle.asn1.pkcs.PKCSObjectIdentifiers;
 import org.bouncycastle.asn1.x500.X500Name;
+import org.bouncycastle.asn1.x509.Extension;
+import org.bouncycastle.asn1.x509.Extensions;
 import org.bouncycastle.cert.crmf.CRMFException;
+import org.bouncycastle.operator.OperatorCreationException;
+import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder;
 import org.bouncycastle.pkcs.jcajce.JcaPKCS10CertificationRequest;
+import org.bouncycastle.pkcs.jcajce.JcaPKCS10CertificationRequestBuilder;
 import org.eclipse.jetty.server.Request;
 import org.eclipse.jetty.server.Response;
 import org.junit.After;
@@ -35,29 +68,6 @@ import org.junit.runner.RunWith;
 import org.mockito.Mock;
 import org.mockito.junit.MockitoJUnitRunner;
 
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringReader;
-import java.io.StringWriter;
-import java.nio.charset.StandardCharsets;
-import java.security.GeneralSecurityException;
-import java.security.InvalidKeyException;
-import java.security.KeyPair;
-import java.security.NoSuchAlgorithmException;
-import java.security.NoSuchProviderException;
-import java.security.cert.X509Certificate;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.ArgumentMatchers.anyInt;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
 @RunWith(MockitoJUnitRunner.class)
 public class TlsCertificateAuthorityServiceHandlerTest {
     X509Certificate caCert;
@@ -95,6 +105,9 @@ public class TlsCertificateAuthorityServiceHandlerTest {
     private String requestedDn;
     private KeyPair certificateKeyPair;
 
+    private static final String SUBJECT_DN = "CN=NiFi Test Server,OU=Security,O=Apache,ST=CA,C=US";
+    private static final List<String> SUBJECT_ALT_NAMES = Arrays.asList("127.0.0.1", "nifi.nifi.apache.org");
+
     @Before
     public void setup() throws Exception {
         testToken = "testTokenTestToken";
@@ -166,6 +179,35 @@ public class TlsCertificateAuthorityServiceHandlerTest {
         tlsCertificateAuthorityServiceHandler.handle(null, baseRequest, httpServletRequest, httpServletResponse);
     }
 
+    @Test
+    public void testSANAgainUsingCertificationRequestMethod() throws GeneralSecurityException, IOException, OperatorCreationException {
+        // Arrange
+        KeyPairGenerator generator = KeyPairGenerator.getInstance("RSA");
+        KeyPair keyPair = generator.generateKeyPair();
+        Extensions exts = TlsHelper.createDomainAlternativeNamesExtensions(SUBJECT_ALT_NAMES, SUBJECT_DN);
+        String token = "someTokenData16B";
+
+        JcaPKCS10CertificationRequestBuilder jcaPKCS10CertificationRequestBuilder = new JcaPKCS10CertificationRequestBuilder(new X500Name(SUBJECT_DN), keyPair.getPublic());
+        jcaPKCS10CertificationRequestBuilder.addAttribute(PKCSObjectIdentifiers.pkcs_9_at_extensionRequest, exts);
+        JcaContentSignerBuilder jcaContentSignerBuilder = new JcaContentSignerBuilder("SHA256WITHRSA");
+        JcaPKCS10CertificationRequest jcaPKCS10CertificationRequest = new JcaPKCS10CertificationRequest(
+                jcaPKCS10CertificationRequestBuilder.build(jcaContentSignerBuilder.build(keyPair.getPrivate())));
+        TlsCertificateAuthorityRequest tlsCertificateAuthorityRequest = new TlsCertificateAuthorityRequest(
+                TlsHelper.calculateHMac(token, jcaPKCS10CertificationRequest.getPublicKey()), TlsHelper.pemEncodeJcaObject(jcaPKCS10CertificationRequest));
+
+        JcaPKCS10CertificationRequest jcaPKCS10CertificationDecoded = TlsHelper.parseCsr(tlsCertificateAuthorityRequest.getCsr());
+
+        // Act
+        Extensions extensions = CertificateUtils.getExtensionsFromCSR(jcaPKCS10CertificationDecoded);
+        // Satisfy @After requirement
+        baseRequest.setHandled(true);
+
+        // Assert
+        assertNotNull("The extensions parsed from the CSR were found to be null when they should have been present.", extensions);
+        assertNotNull("The Subject Alternate Name parsed from the CSR was found to be null when it should have been present.", extensions.getExtension(Extension.subjectAlternativeName));
+        assertTrue(extensions.equivalent(exts));
+    }
+
     @After
     public void verifyHandled() {
         verify(baseRequest).setHandled(true);


[nifi] 10/16: NIFI-7762 - support copy-paste on Disabled ports

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.12.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 87d65e7255eb1205d3d3902f3ed7b810072a17a3
Author: jmconte <je...@atos.net>
AuthorDate: Tue Aug 25 17:06:43 2020 +0200

    NIFI-7762 - support copy-paste on Disabled ports
---
 .../src/main/java/org/apache/nifi/connectable/Port.java      |  2 ++
 .../main/java/org/apache/nifi/controller/AbstractPort.java   |  1 +
 .../java/org/apache/nifi/controller/StandardFlowSnippet.java |  6 ++++++
 .../src/main/java/org/apache/nifi/web/util/SnippetUtils.java | 12 ++++++++++--
 4 files changed, 19 insertions(+), 2 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/Port.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/Port.java
index 907dd92..f04038e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/Port.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/Port.java
@@ -28,4 +28,6 @@ public interface Port extends Connectable {
      * Port a chance to initialize any resources needed.</p>
      */
     void onSchedulingStart();
+
+    void disable();
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java
index 9af1bd1..dcbb21e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java
@@ -423,6 +423,7 @@ public abstract class AbstractPort implements Port {
         scheduledState.set(ScheduledState.RUNNING);
     }
 
+    @Override
     public void disable() {
         final boolean updated = scheduledState.compareAndSet(ScheduledState.STOPPED, ScheduledState.DISABLED);
         if (!updated) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSnippet.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSnippet.java
index 85e5984..215ec89 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSnippet.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSnippet.java
@@ -308,6 +308,9 @@ public class StandardFlowSnippet implements FlowSnippet {
             inputPort.setProcessGroup(group);
             inputPort.setMaxConcurrentTasks(portDTO.getConcurrentlySchedulableTaskCount());
             inputPort.setComments(portDTO.getComments());
+            if (portDTO.getState().equals(ScheduledState.DISABLED.toString())) {
+                inputPort.disable();
+            }
             group.addInputPort(inputPort);
         }
 
@@ -333,6 +336,9 @@ public class StandardFlowSnippet implements FlowSnippet {
             outputPort.setProcessGroup(group);
             outputPort.setMaxConcurrentTasks(portDTO.getConcurrentlySchedulableTaskCount());
             outputPort.setComments(portDTO.getComments());
+            if (portDTO.getState().equals(ScheduledState.DISABLED.toString())) {
+                outputPort.disable();
+            }
             group.addOutputPort(outputPort);
         }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java
index 7e81c9b..00bbedc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java
@@ -551,7 +551,11 @@ public final class SnippetUtils {
                     final PortDTO cp = dtoFactory.copy(portDTO);
                     cp.setId(generateId(portDTO.getId(), idGenerationSeed, isCopy));
                     cp.setParentGroupId(groupId);
-                    cp.setState(ScheduledState.STOPPED.toString());
+                    if (portDTO.getState() != null && portDTO.getState().equals(ScheduledState.DISABLED.toString())) {
+                        cp.setState(ScheduledState.DISABLED.toString());
+                    } else {
+                        cp.setState(ScheduledState.STOPPED.toString());
+                    }
                     inputPorts.add(cp);
 
                     final ConnectableDTO portConnectable = dtoFactory.createConnectableDto(cp, ConnectableType.INPUT_PORT);
@@ -576,7 +580,11 @@ public final class SnippetUtils {
                     final PortDTO cp = dtoFactory.copy(portDTO);
                     cp.setId(generateId(portDTO.getId(), idGenerationSeed, isCopy));
                     cp.setParentGroupId(groupId);
-                    cp.setState(ScheduledState.STOPPED.toString());
+                    if (portDTO.getState() != null && portDTO.getState().equals(ScheduledState.DISABLED.toString())) {
+                        cp.setState(ScheduledState.DISABLED.toString());
+                    } else {
+                        cp.setState(ScheduledState.STOPPED.toString());
+                    }
                     outputPorts.add(cp);
 
                     final ConnectableDTO portConnectable = dtoFactory.createConnectableDto(cp, ConnectableType.OUTPUT_PORT);


[nifi] 15/16: NIFI-7730 Added regression tests for multiple certificate keystores. Cleaned up JettyServer code. Changed test logging severity to include debug statements. Added test resources.

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.12.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 29fc0a7c9d4652f40eeabaec57676dbb115c84d8
Author: Kotaro Terada <ko...@apache.org>
AuthorDate: Tue Aug 11 18:19:23 2020 +0900

    NIFI-7730 Added regression tests for multiple certificate keystores.
    Cleaned up JettyServer code.
    Changed test logging severity to include debug statements.
    Added test resources.
    
    This closes #4498.
    
    Co-authored-by: Kotaro Terada <ko...@apache.org>
---
 .../nifi/remote/client/http/TestHttpClient.java    |   2 +-
 .../org/apache/nifi/web/server/JettyServer.java    |  14 +--
 .../nifi/web/server/JettyServerGroovyTest.groovy   | 105 +++++++++++++++++----
 .../apache/nifi/web/server/JettyServerTest.java    |  56 +++++------
 .../nifi-jetty/src/test/resources/log4j.properties |   1 +
 .../src/test/resources/multiple_cert_keystore.jks  | Bin 0 -> 4457 bytes
 .../src/test/resources/multiple_cert_keystore.p12  | Bin 0 -> 4949 bytes
 .../nifi/integration/util/NiFiTestServer.java      |   7 +-
 .../reporting/prometheus/PrometheusServer.java     |   2 +-
 .../processors/standard/HandleHttpRequest.java     |   7 +-
 .../nifi/processors/standard/ListenHTTP.java       |   7 +-
 .../processors/standard/TestGetHTTPGroovy.groovy   |   4 +-
 .../processors/standard/TestPostHTTPGroovy.groovy  |   4 +-
 .../java/org/apache/nifi/web/util/TestServer.java  |   7 +-
 .../jetty/AbstractJettyWebSocketService.java       |   2 +-
 .../websocket/example/WebSocketClientExample.java  |   2 +-
 .../websocket/example/WebSocketServerExample.java  |   7 +-
 .../server/TlsCertificateAuthorityService.java     |   7 +-
 18 files changed, 136 insertions(+), 98 deletions(-)

diff --git a/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/http/TestHttpClient.java b/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/http/TestHttpClient.java
index f6bf811..ab71c56 100644
--- a/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/http/TestHttpClient.java
+++ b/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/http/TestHttpClient.java
@@ -453,7 +453,7 @@ public class TestHttpClient {
         final ServletHandler wrongPathServletHandler = new ServletHandler();
         wrongPathContextHandler.insertHandler(wrongPathServletHandler);
 
-        final SslContextFactory sslContextFactory = new SslContextFactory();
+        final SslContextFactory sslContextFactory = new SslContextFactory.Server();
         sslContextFactory.setKeyStorePath("src/test/resources/certs/keystore.jks");
         sslContextFactory.setKeyStorePassword("passwordpassword");
         sslContextFactory.setKeyStoreType("JKS");
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
index ca7944f..e53c785 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
@@ -975,19 +975,13 @@ public class JettyServer implements NiFiServer, ExtensionUiLoader {
     }
 
     private SslContextFactory createSslContextFactory() {
-        final SslContextFactory contextFactory = new SslContextFactory();
-        configureSslContextFactory(contextFactory, props);
-        return contextFactory;
+        final SslContextFactory.Server serverContextFactory = new SslContextFactory.Server();
+        configureSslContextFactory(serverContextFactory, props);
+        return serverContextFactory;
     }
 
-    protected static void configureSslContextFactory(SslContextFactory contextFactory, NiFiProperties props) {
-        // Need to set SslContextFactory's endpointIdentificationAlgorithm to null; this is a server,
-        // not a client.  Server does not need to perform hostname verification on the client.
-        // Previous to Jetty 9.4.15.v20190215, this defaulted to null, and now defaults to "HTTPS".
-        contextFactory.setEndpointIdentificationAlgorithm(null);
-
+    protected static void configureSslContextFactory(SslContextFactory.Server contextFactory, NiFiProperties props) {
         // Explicitly exclude legacy TLS protocol versions
-        // contextFactory.setProtocol(CertificateUtils.getHighestCurrentSupportedTlsProtocolVersion());
         contextFactory.setIncludeProtocols(CertificateUtils.getCurrentSupportedTlsProtocolVersions());
         contextFactory.setExcludeProtocols("TLS", "TLSv1", "TLSv1.1", "SSL", "SSLv2", "SSLv2Hello", "SSLv3");
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/groovy/org/apache/nifi/web/server/JettyServerGroovyTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/groovy/org/apache/nifi/web/server/JettyServerGroovyTest.groovy
index 64077a0..054ad08 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/groovy/org/apache/nifi/web/server/JettyServerGroovyTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/groovy/org/apache/nifi/web/server/JettyServerGroovyTest.groovy
@@ -19,6 +19,9 @@ package org.apache.nifi.web.server
 import org.apache.log4j.AppenderSkeleton
 import org.apache.log4j.spi.LoggingEvent
 import org.apache.nifi.bundle.Bundle
+import org.apache.nifi.nar.ExtensionManagerHolder
+import org.apache.nifi.nar.ExtensionMapping
+import org.apache.nifi.nar.SystemBundle
 import org.apache.nifi.processor.DataUnit
 import org.apache.nifi.properties.StandardNiFiProperties
 import org.apache.nifi.security.util.CertificateUtils
@@ -121,6 +124,8 @@ class JettyServerGroovyTest extends GroovyTestCase {
 
     @After
     void tearDown() throws Exception {
+        // Cleans up the EMH so it can be reinitialized when a new Jetty server starts
+        ExtensionManagerHolder.INSTANCE = null
         TestAppender.reset()
     }
 
@@ -196,10 +201,9 @@ class JettyServerGroovyTest extends GroovyTestCase {
         assert !bothConfigsPresentForHttp
         assert !bothConfigsPresentForHttps
 
-        // Verifies that the warning was not logged
-        assert log.size() == 2
-        assert log.first() == "Both configs present for HTTP properties: false"
-        assert log.last() == "Both configs present for HTTPS properties: false"
+        // Verifies that the warning was not logged (messages are duplicated because of log4j.properties settings)
+        assert log.size() == 4
+        assert log.every { it =~ "Both configs present for HTTPS? properties: false" }
     }
 
     @Test
@@ -242,10 +246,84 @@ class JettyServerGroovyTest extends GroovyTestCase {
         // Assertions defined above
     }
 
+    /**
+     * Regression test added after NiFi 1.12.0 because Jetty upgrade to 9.4.26 no longer works
+     * with multiple certificate keystores.
+     */
+    @Test
+    void testShouldStartWithMultipleCertificatePKCS12Keystore() {
+        // Arrange
+        final String externalHostname = "localhost"
+
+        NiFiProperties httpsProps = new StandardNiFiProperties(rawProperties: new Properties([
+                (NiFiProperties.WEB_HTTPS_PORT): HTTPS_PORT as String,
+                (NiFiProperties.WEB_HTTPS_HOST): externalHostname,
+                (NiFiProperties.SECURITY_KEYSTORE): "src/test/resources/multiple_cert_keystore.p12",
+                (NiFiProperties.SECURITY_KEYSTORE_PASSWD): "passwordpassword",
+                (NiFiProperties.SECURITY_KEYSTORE_TYPE): "PKCS12",
+                (NiFiProperties.NAR_LIBRARY_DIRECTORY): "target/"
+        ]))
+
+        JettyServer jetty = createJettyServer(httpsProps)
+        Server internalServer = jetty.server
+        List<Connector> connectors = Arrays.asList(internalServer.connectors)
+
+        // Act
+        jetty.start()
+
+        // Assert
+        assertServerConnector(connectors, "TLS", CURRENT_TLS_PROTOCOL_VERSIONS, CURRENT_TLS_PROTOCOL_VERSIONS, externalHostname, HTTPS_PORT)
+
+        // Clean up
+        jetty.stop()
+    }
+
+    /**
+     * Regression test added after NiFi 1.12.0 because Jetty upgrade to 9.4.26 no longer works
+     * with multiple certificate keystores.
+     */
+    @Test
+    void testShouldStartWithMultipleCertificateJKSKeystore() {
+        // Arrange
+        final String externalHostname = "localhost"
+
+        NiFiProperties httpsProps = new StandardNiFiProperties(rawProperties: new Properties([
+                (NiFiProperties.WEB_HTTPS_PORT): HTTPS_PORT as String,
+                (NiFiProperties.WEB_HTTPS_HOST): externalHostname,
+                (NiFiProperties.SECURITY_KEYSTORE): "src/test/resources/multiple_cert_keystore.jks",
+                (NiFiProperties.SECURITY_KEYSTORE_PASSWD): "passwordpassword",
+                (NiFiProperties.SECURITY_KEYSTORE_TYPE): "JKS",
+                (NiFiProperties.NAR_LIBRARY_DIRECTORY): "target/"
+        ]))
+
+        JettyServer jetty = createJettyServer(httpsProps)
+        Server internalServer = jetty.server
+        List<Connector> connectors = Arrays.asList(internalServer.connectors)
+
+        // Act
+        jetty.start()
+
+        // Assert
+        assertServerConnector(connectors, "TLS", CURRENT_TLS_PROTOCOL_VERSIONS, CURRENT_TLS_PROTOCOL_VERSIONS, externalHostname, HTTPS_PORT)
+
+        // Clean up
+        jetty.stop()
+    }
+
+    private static JettyServer createJettyServer(StandardNiFiProperties httpsProps) {
+        Server internalServer = new Server()
+        JettyServer jetty = new JettyServer(internalServer, httpsProps)
+        jetty.systemBundle = SystemBundle.create(httpsProps)
+        jetty.bundles = [] as Set<Bundle>
+        jetty.extensionMapping = [size: { -> 0 }] as ExtensionMapping
+        jetty.configureHttpsConnector(internalServer, new HttpConfiguration())
+        jetty
+    }
+
     @Test
     void testShouldConfigureHTTPSConnector() {
         // Arrange
-        final String externalHostname = "secure.host.com"
+        final String externalHostname = "localhost"
 
         NiFiProperties httpsProps = new StandardNiFiProperties(rawProperties: new Properties([
                 (NiFiProperties.WEB_HTTPS_PORT): HTTPS_PORT as String,
@@ -260,9 +338,7 @@ class JettyServerGroovyTest extends GroovyTestCase {
         List<Connector> connectors = Arrays.asList(internalServer.connectors)
 
         // Assert
-
-        // Set the expected TLS protocols to null because no actual keystore/truststore is loaded here
-        assertServerConnector(connectors, "TLS", null, null, externalHostname, HTTPS_PORT)
+        assertServerConnector(connectors, "TLS", CURRENT_TLS_PROTOCOL_VERSIONS, CURRENT_TLS_PROTOCOL_VERSIONS, externalHostname, HTTPS_PORT)
     }
 
     @Test
@@ -411,16 +487,13 @@ class JettyServerGroovyTest extends GroovyTestCase {
         assert connector.port == EXPECTED_PORT
         assert connector.getProtocols() == ['ssl', 'http/1.1']
 
-        // This kind of testing is not ideal as it breaks encapsulation, but is necessary to enforce verification of the TLS protocol versions specified
         SslConnectionFactory connectionFactory = connector.getConnectionFactory("ssl") as SslConnectionFactory
-        SslContextFactory sslContextFactory = connectionFactory._sslContextFactory as SslContextFactory
+        SslContextFactory sslContextFactory = connectionFactory.getSslContextFactory()
         logger.debug("SSL Context Factory: ${sslContextFactory.dump()}")
 
-        // Using the getters is subject to NPE due to blind array copies
-        assert sslContextFactory._sslProtocol == EXPECTED_TLS_PROTOCOL
-        assert sslContextFactory._includeProtocols.containsAll(EXPECTED_INCLUDED_PROTOCOLS ?: Collections.emptySet())
-        assert (sslContextFactory._excludeProtocols as List<String>).containsAll(LEGACY_TLS_PROTOCOLS)
-        assert sslContextFactory._selectedProtocols == EXPECTED_SELECTED_PROTOCOLS as String[]
+        assert sslContextFactory.getProtocol() == EXPECTED_TLS_PROTOCOL
+        assert Arrays.asList(sslContextFactory.getIncludeProtocols()).containsAll(EXPECTED_INCLUDED_PROTOCOLS ?: Collections.emptySet())
+        assert (sslContextFactory.getExcludeProtocols() as List<String>).containsAll(LEGACY_TLS_PROTOCOLS)
     }
 
     @Test
@@ -531,4 +604,4 @@ class TestAppender extends AppenderSkeleton {
             events.collect { LoggingEvent le -> le.getRenderedMessage() }
         }
     }
-}
\ No newline at end of file
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/java/org/apache/nifi/web/server/JettyServerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/java/org/apache/nifi/web/server/JettyServerTest.java
index d25980a..63e9ad5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/java/org/apache/nifi/web/server/JettyServerTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/java/org/apache/nifi/web/server/JettyServerTest.java
@@ -43,12 +43,12 @@ public class JettyServerTest {
         addProps.put(NiFiProperties.SECURITY_KEYSTORE_PASSWD, testKeystorePassword);
         addProps.put(NiFiProperties.SECURITY_KEY_PASSWD, testKeyPassword);
         NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(null, addProps);
-        SslContextFactory contextFactory = mock(SslContextFactory.class);
+        SslContextFactory.Server mockSCF = mock(SslContextFactory.Server.class);
 
-        JettyServer.configureSslContextFactory(contextFactory, nifiProperties);
+        JettyServer.configureSslContextFactory(mockSCF, nifiProperties);
 
-        verify(contextFactory).setKeyStorePassword(testKeystorePassword);
-        verify(contextFactory).setKeyManagerPassword(testKeyPassword);
+        verify(mockSCF).setKeyStorePassword(testKeystorePassword);
+        verify(mockSCF).setKeyManagerPassword(testKeyPassword);
     }
 
     @Test
@@ -59,12 +59,12 @@ public class JettyServerTest {
         final Map<String, String> addProps = new HashMap<>();
         addProps.put(NiFiProperties.SECURITY_KEY_PASSWD, testKeyPassword);
         NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(null, addProps);
-        SslContextFactory contextFactory = mock(SslContextFactory.class);
+        SslContextFactory.Server mockSCF = mock(SslContextFactory.Server.class);
 
-        JettyServer.configureSslContextFactory(contextFactory, nifiProperties);
+        JettyServer.configureSslContextFactory(mockSCF, nifiProperties);
 
-        verify(contextFactory).setKeyManagerPassword(testKeyPassword);
-        verify(contextFactory, never()).setKeyStorePassword(anyString());
+        verify(mockSCF).setKeyManagerPassword(testKeyPassword);
+        verify(mockSCF, never()).setKeyStorePassword(anyString());
     }
 
     @Test
@@ -75,12 +75,12 @@ public class JettyServerTest {
         final Map<String, String> addProps = new HashMap<>();
         addProps.put(NiFiProperties.SECURITY_KEYSTORE_PASSWD, testKeystorePassword);
         NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(null, addProps);
-        SslContextFactory contextFactory = mock(SslContextFactory.class);
+        SslContextFactory.Server mockSCF = mock(SslContextFactory.Server.class);
 
-        JettyServer.configureSslContextFactory(contextFactory, nifiProperties);
+        JettyServer.configureSslContextFactory(mockSCF, nifiProperties);
 
-        verify(contextFactory).setKeyStorePassword(testKeystorePassword);
-        verify(contextFactory).setKeyManagerPassword(testKeystorePassword);
+        verify(mockSCF).setKeyStorePassword(testKeystorePassword);
+        verify(mockSCF).setKeyManagerPassword(testKeystorePassword);
     }
 
     @Test
@@ -90,12 +90,12 @@ public class JettyServerTest {
         String keyStoreType = KeystoreType.JKS.toString();
         addProps.put(NiFiProperties.SECURITY_KEYSTORE_TYPE, keyStoreType);
         NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(null, addProps);
-        SslContextFactory contextFactory = mock(SslContextFactory.class);
+        SslContextFactory.Server mockSCF = mock(SslContextFactory.Server.class);
 
-        JettyServer.configureSslContextFactory(contextFactory, nifiProperties);
+        JettyServer.configureSslContextFactory(mockSCF, nifiProperties);
 
-        verify(contextFactory).setKeyStoreType(keyStoreType);
-        verify(contextFactory).setKeyStoreProvider(SUN_PROVIDER_NAME);
+        verify(mockSCF).setKeyStoreType(keyStoreType);
+        verify(mockSCF).setKeyStoreProvider(SUN_PROVIDER_NAME);
     }
 
     @Test
@@ -105,12 +105,12 @@ public class JettyServerTest {
         String keyStoreType = KeystoreType.PKCS12.toString();
         addProps.put(NiFiProperties.SECURITY_KEYSTORE_TYPE, keyStoreType);
         NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(null, addProps);
-        SslContextFactory contextFactory = mock(SslContextFactory.class);
+        SslContextFactory.Server mockSCF = mock(SslContextFactory.Server.class);
 
-        JettyServer.configureSslContextFactory(contextFactory, nifiProperties);
+        JettyServer.configureSslContextFactory(mockSCF, nifiProperties);
 
-        verify(contextFactory).setKeyStoreType(keyStoreType);
-        verify(contextFactory).setKeyStoreProvider(BouncyCastleProvider.PROVIDER_NAME);
+        verify(mockSCF).setKeyStoreType(keyStoreType);
+        verify(mockSCF).setKeyStoreProvider(BouncyCastleProvider.PROVIDER_NAME);
     }
 
     @Test
@@ -120,12 +120,12 @@ public class JettyServerTest {
         String trustStoreType = KeystoreType.JKS.toString();
         addProps.put(NiFiProperties.SECURITY_TRUSTSTORE_TYPE, trustStoreType);
         NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(null, addProps);
-        SslContextFactory contextFactory = mock(SslContextFactory.class);
+        SslContextFactory.Server mockSCF = mock(SslContextFactory.Server.class);
 
-        JettyServer.configureSslContextFactory(contextFactory, nifiProperties);
+        JettyServer.configureSslContextFactory(mockSCF, nifiProperties);
 
-        verify(contextFactory).setTrustStoreType(trustStoreType);
-        verify(contextFactory).setTrustStoreProvider(SUN_PROVIDER_NAME);
+        verify(mockSCF).setTrustStoreType(trustStoreType);
+        verify(mockSCF).setTrustStoreProvider(SUN_PROVIDER_NAME);
     }
 
     @Test
@@ -135,11 +135,11 @@ public class JettyServerTest {
         String trustStoreType = KeystoreType.PKCS12.toString();
         addProps.put(NiFiProperties.SECURITY_TRUSTSTORE_TYPE, trustStoreType);
         NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(null, addProps);
-        SslContextFactory contextFactory = mock(SslContextFactory.class);
+        SslContextFactory.Server mockSCF = mock(SslContextFactory.Server.class);
 
-        JettyServer.configureSslContextFactory(contextFactory, nifiProperties);
+        JettyServer.configureSslContextFactory(mockSCF, nifiProperties);
 
-        verify(contextFactory).setTrustStoreType(trustStoreType);
-        verify(contextFactory).setTrustStoreProvider(BouncyCastleProvider.PROVIDER_NAME);
+        verify(mockSCF).setTrustStoreType(trustStoreType);
+        verify(mockSCF).setTrustStoreProvider(BouncyCastleProvider.PROVIDER_NAME);
     }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/log4j.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/log4j.properties
index 162521f..2d5c71c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/log4j.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/log4j.properties
@@ -16,6 +16,7 @@
 #
 
 log4j.rootLogger=INFO,console,test
+log4j.logger.org.apache.nifi.web=DEBUG,console,test
 
 log4j.appender.console=org.apache.log4j.ConsoleAppender
 log4j.appender.console.Target=System.err
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/multiple_cert_keystore.jks b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/multiple_cert_keystore.jks
new file mode 100644
index 0000000..e563d45
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/multiple_cert_keystore.jks differ
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/multiple_cert_keystore.p12 b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/multiple_cert_keystore.p12
new file mode 100644
index 0000000..2bdec71
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/multiple_cert_keystore.p12 differ
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java
index 9379019..e61dbae 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java
@@ -77,12 +77,7 @@ public class NiFiTestServer {
 
     // TODO: Refactor this method to use proper factory methods
     private void createSecureConnector() {
-        org.eclipse.jetty.util.ssl.SslContextFactory contextFactory = new org.eclipse.jetty.util.ssl.SslContextFactory();
-
-        // Need to set SslContextFactory's endpointIdentificationAlgorithm to null; this is a server,
-        // not a client.  Server does not need to perform hostname verification on the client.
-        // Previous to Jetty 9.4.15.v20190215, this defaulted to null, and now defaults to "HTTPS".
-        contextFactory.setEndpointIdentificationAlgorithm(null);
+        org.eclipse.jetty.util.ssl.SslContextFactory contextFactory = new org.eclipse.jetty.util.ssl.SslContextFactory.Server();
 
         // require client auth when not supporting login or anonymous access
         if (StringUtils.isBlank(properties.getProperty(NiFiProperties.SECURITY_USER_LOGIN_IDENTITY_PROVIDER))) {
diff --git a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/src/main/java/org/apache/nifi/reporting/prometheus/PrometheusServer.java b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/src/main/java/org/apache/nifi/reporting/prometheus/PrometheusServer.java
index 50d5ee1..d57f1c1 100644
--- a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/src/main/java/org/apache/nifi/reporting/prometheus/PrometheusServer.java
+++ b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/src/main/java/org/apache/nifi/reporting/prometheus/PrometheusServer.java
@@ -113,7 +113,7 @@ public class PrometheusServer {
     }
 
     private SslContextFactory createSslFactory(final SSLContextService sslService, boolean needClientAuth, boolean wantClientAuth) {
-        SslContextFactory sslFactory = new SslContextFactory();
+        SslContextFactory sslFactory = new SslContextFactory.Server();
 
         sslFactory.setNeedClientAuth(needClientAuth);
         sslFactory.setWantClientAuth(wantClientAuth);
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java
index a7d3c89..dbdb995ea 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java
@@ -520,18 +520,13 @@ public class HandleHttpRequest extends AbstractProcessor {
     }
 
     private SslContextFactory createSslFactory(final SSLContextService sslService, final boolean needClientAuth, final boolean wantClientAuth) {
-        final SslContextFactory sslFactory = new SslContextFactory();
+        final SslContextFactory sslFactory = new SslContextFactory.Server();
 
         sslFactory.setNeedClientAuth(needClientAuth);
         sslFactory.setWantClientAuth(wantClientAuth);
 
         sslFactory.setProtocol(sslService.getSslAlgorithm());
 
-        // Need to set SslContextFactory's endpointIdentificationAlgorithm to null; this is a server,
-        // not a client.  Server does not need to perform hostname verification on the client.
-        // Previous to Jetty 9.4.15.v20190215, this defaulted to null.
-        sslFactory.setEndpointIdentificationAlgorithm(null);
-
         if (sslService.isKeyStoreConfigured()) {
             sslFactory.setKeyStorePath(sslService.getKeyStoreFile());
             sslFactory.setKeyStorePassword(sslService.getKeyStorePassword());
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
index af17a6e..cfc1a2a 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
@@ -256,14 +256,9 @@ public class ListenHTTP extends AbstractSessionFactoryProcessor {
 
         final boolean needClientAuth = sslContextService != null && sslContextService.getTrustStoreFile() != null;
 
-        final SslContextFactory contextFactory = new SslContextFactory();
+        final SslContextFactory contextFactory = new SslContextFactory.Server();
         contextFactory.setNeedClientAuth(needClientAuth);
 
-        // Need to set SslContextFactory's endpointIdentificationAlgorithm to null; this is a server,
-        // not a client.  Server does not need to perform hostname verification on the client.
-        // Previous to Jetty 9.4.15.v20190215, this defaulted to null, and now defaults to "HTTPS".
-        contextFactory.setEndpointIdentificationAlgorithm(null);
-
         if (needClientAuth) {
             contextFactory.setTrustStorePath(sslContextService.getTrustStoreFile());
             contextFactory.setTrustStoreType(sslContextService.getTrustStoreType());
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/TestGetHTTPGroovy.groovy b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/TestGetHTTPGroovy.groovy
index e8266f5..a01874d 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/TestGetHTTPGroovy.groovy
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/TestGetHTTPGroovy.groovy
@@ -141,7 +141,7 @@ class TestGetHTTPGroovy extends GroovyTestCase {
 
     private
     static SslContextFactory createSslContextFactory(List supportedProtocols = DEFAULT_PROTOCOLS, List supportedCipherSuites = DEFAULT_CIPHER_SUITES) {
-        final SslContextFactory contextFactory = new SslContextFactory()
+        final SslContextFactory contextFactory = new SslContextFactory.Server()
         contextFactory.needClientAuth = false
         contextFactory.wantClientAuth = false
 
@@ -485,4 +485,4 @@ class TestGetHTTPGroovy extends GroovyTestCase {
         runner.clearTransferState()
         logger.info("Ran successfully")
     }
-}
\ No newline at end of file
+}
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/TestPostHTTPGroovy.groovy b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/TestPostHTTPGroovy.groovy
index be38cd3..8b96bdc 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/TestPostHTTPGroovy.groovy
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/TestPostHTTPGroovy.groovy
@@ -133,7 +133,7 @@ class TestPostHTTPGroovy extends GroovyTestCase {
 
     private
     static SslContextFactory createSslContextFactory(List supportedProtocols = DEFAULT_PROTOCOLS, List supportedCipherSuites = DEFAULT_CIPHER_SUITES) {
-        final SslContextFactory contextFactory = new SslContextFactory()
+        final SslContextFactory contextFactory = new SslContextFactory.Server()
         contextFactory.needClientAuth = false
         contextFactory.wantClientAuth = false
 
@@ -440,4 +440,4 @@ class TestPostHTTPGroovy extends GroovyTestCase {
         runner.clearTransferState()
         logger.info("Ran successfully")
     }
-}
\ No newline at end of file
+}
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-web-test-utils/src/main/java/org/apache/nifi/web/util/TestServer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-web-test-utils/src/main/java/org/apache/nifi/web/util/TestServer.java
index 23e128b..e691d6b 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-web-test-utils/src/main/java/org/apache/nifi/web/util/TestServer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-web-test-utils/src/main/java/org/apache/nifi/web/util/TestServer.java
@@ -76,7 +76,7 @@ public class TestServer {
     }
 
     private void createSecureConnector(final Map<String, String> sslProperties) {
-        SslContextFactory ssl = new SslContextFactory();
+        SslContextFactory ssl = new SslContextFactory.Server();
 
         if (sslProperties.get(StandardSSLContextService.KEYSTORE.getName()) != null) {
             ssl.setKeyStorePath(sslProperties.get(StandardSSLContextService.KEYSTORE.getName()));
@@ -97,11 +97,6 @@ public class TestServer {
             ssl.setNeedClientAuth(Boolean.parseBoolean(clientAuth));
         }
 
-        // Need to set SslContextFactory's endpointIdentificationAlgorithm to null; this is a server,
-        // not a client.  Server does not need to perform hostname verification on the client.
-        // Previous to Jetty 9.4.15.v20190215, this defaulted to null, and now defaults to "HTTPS".
-        ssl.setEndpointIdentificationAlgorithm(null);
-
         // build the connector
         final ServerConnector https = new ServerConnector(jetty, ssl);
 
diff --git a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/AbstractJettyWebSocketService.java b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/AbstractJettyWebSocketService.java
index 546a853..abff785 100644
--- a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/AbstractJettyWebSocketService.java
+++ b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/main/java/org/apache/nifi/websocket/jetty/AbstractJettyWebSocketService.java
@@ -67,7 +67,7 @@ public abstract class AbstractJettyWebSocketService extends AbstractWebSocketSer
 
 
     protected SslContextFactory createSslFactory(final SSLContextService sslService, final boolean needClientAuth, final boolean wantClientAuth, final String endpointIdentificationAlgorithm) {
-        final SslContextFactory sslFactory = new SslContextFactory();
+        final SslContextFactory sslFactory = new SslContextFactory.Server();
 
         sslFactory.setNeedClientAuth(needClientAuth);
         sslFactory.setWantClientAuth(wantClientAuth);
diff --git a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/example/WebSocketClientExample.java b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/example/WebSocketClientExample.java
index f8c9b3a..f222bc2 100644
--- a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/example/WebSocketClientExample.java
+++ b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/example/WebSocketClientExample.java
@@ -46,7 +46,7 @@ public class WebSocketClientExample {
         String destUri = "wss://localhost:50010/test";
 
         final CountDownLatch replyLatch = new CountDownLatch(1);
-        final SslContextFactory sslContextFactory = new SslContextFactory();
+        final SslContextFactory sslContextFactory = new SslContextFactory.Client();
         sslContextFactory.setKeyStorePath("src/test/resources/certs/keystore.jks");
         sslContextFactory.setKeyStorePassword("passwordpassword");
         sslContextFactory.setKeyStoreType("JKS");
diff --git a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/example/WebSocketServerExample.java b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/example/WebSocketServerExample.java
index dad8520..913031a 100644
--- a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/example/WebSocketServerExample.java
+++ b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/example/WebSocketServerExample.java
@@ -152,16 +152,11 @@ public class WebSocketServerExample {
         httpConnector = new ServerConnector(server);
         httpConnector.setPort(50010);
 
-        final SslContextFactory sslContextFactory = new SslContextFactory();
+        final SslContextFactory sslContextFactory = new SslContextFactory.Server();
         sslContextFactory.setKeyStorePath("src/test/resources/certs/keystore.jks");
         sslContextFactory.setKeyStorePassword("passwordpassword");
         sslContextFactory.setKeyStoreType("JKS");
 
-        // Need to set SslContextFactory's endpointIdentificationAlgorithm to null; this is a server,
-        // not a client.  Server does not need to perform hostname verification on the client.
-        // Previous to Jetty 9.4.15.v20190215, this defaulted to null, and now defaults to "HTTPS".
-        sslContextFactory.setEndpointIdentificationAlgorithm(null);
-
         final HttpConfiguration https = new HttpConfiguration();
         https.addCustomizer(new SecureRequestCustomizer());
         sslConnector = new ServerConnector(server,
diff --git a/nifi-toolkit/nifi-toolkit-tls/src/main/java/org/apache/nifi/toolkit/tls/service/server/TlsCertificateAuthorityService.java b/nifi-toolkit/nifi-toolkit-tls/src/main/java/org/apache/nifi/toolkit/tls/service/server/TlsCertificateAuthorityService.java
index 7b6d1e2..d95ae8e 100644
--- a/nifi-toolkit/nifi-toolkit-tls/src/main/java/org/apache/nifi/toolkit/tls/service/server/TlsCertificateAuthorityService.java
+++ b/nifi-toolkit/nifi-toolkit-tls/src/main/java/org/apache/nifi/toolkit/tls/service/server/TlsCertificateAuthorityService.java
@@ -62,16 +62,11 @@ public class TlsCertificateAuthorityService {
     private static Server createServer(Handler handler, int port, KeyStore keyStore, String keyPassword) throws Exception {
         Server server = new Server();
 
-        SslContextFactory sslContextFactory = new SslContextFactory();
+        SslContextFactory sslContextFactory = new SslContextFactory.Server();
         sslContextFactory.setIncludeProtocols(CertificateUtils.getHighestCurrentSupportedTlsProtocolVersion());
         sslContextFactory.setKeyStore(keyStore);
         sslContextFactory.setKeyManagerPassword(keyPassword);
 
-        // Need to set SslContextFactory's endpointIdentificationAlgorithm to null; this is a server,
-        // not a client.  Server does not need to perform hostname verification on the client.
-        // Previous to Jetty 9.4.15.v20190215, this defaulted to null, and now defaults to "HTTPS".
-        sslContextFactory.setEndpointIdentificationAlgorithm(null);
-
         HttpConfiguration httpsConfig = new HttpConfiguration();
         httpsConfig.addCustomizer(new SecureRequestCustomizer());