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 2022/11/30 19:53:38 UTC

[nifi] branch support/nifi-1.19 updated (ff9ae733d6 -> eb0d07fda3)

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

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


    from ff9ae733d6 NIFI-10913 updating all to 1.19.1-snapshot.  line builds and ready
     new 3caad988be NIFI-10866 Refactored Kafka 1.0 and 2.0 using nifi-kafka-shared
     new 2834aab432 NIFI-10850: Fixed Query Parameters property in InvokeAWSGatewayApi should support FlowFile attributes in EL
     new 0a09aa57fd NIFI-6428 Add 'inTransaction' value to state,fix bug
     new 51395bddd1 NIFI-10845 - JsonQueryElasticsearch processors are not outputting an empty flow file for a combined response with output_no_hits set to true
     new db2bf5f645 NIFI-5819: Support SQLServer sql_variant type
     new e0b4feac57 NIFI-10873 - GenerateFlowFile: flowfiles in a batch are not unique
     new 69e4786666 NIFI-10874 Fixed issue with multiple levels of versioned flows
     new bfa59b2041 NIFI-10872 Fix broken Java Mail links in ConsumeIMAP and ConsumePOP3 documentation
     new 83d9f9916a NIFI-10886 Upgraded PostgreSQL JDBC Driver from 42.4.1 to 42.4.3
     new 0ed8114dcb NIFI-7190 CaptureChangeMySQL processor remove comments from normalized query
     new d502831559 NIFI-10882 Set credentials for ElasticSearchClientService based on AuthorizationScheme
     new 2e6f8ecbe0 NIFI-10875 Changed TestQuerySolr to check for path elements
     new 03933c6425 NIFI-10834 Updated TestPutSplunkHTTP to avoid non-deterministic tests
     new ec813fc445 NIFI-10859 Upgraded metrics-jvm from 4.1.2 to 4.1.33
     new 4aa86b850d NIFI-10871 Skipped CSRF processing for replicated HTTP requests
     new 71f41b96ba NIFI-10785 Allow publishing AMQP message with null header value NIFI-10785 addressing review comment NIFI-10785 addressing review comments (remove unnecessary property to ignore null headers)
     new e6dfea6352 NIFI-10890 Replaced HashMap with LinkedHashMap in JsonRowRecordReader
     new 8c74f54b9d NIFI-10891 Replaced HashMap with LinkedHashMap in BinFiles
     new c36c4c0bbc NIFI-10765 Added better error reporting in JASN1Reader
     new 8e0eeded00 NIFI-10910 Upgraded amqp-client from 5.15.0 to 5.16.0
     new 6808a5944d NIFI-10901 - PublishKafka headers not sent in ProducerRecord (#6731)
     new eb0d07fda3 NIFI-10913 updating versions post commit landing

The 22 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.


Summary of changes:
 nifi-commons/nifi-metrics/pom.xml                  |   4 +-
 .../nifi-amqp-bundle/nifi-amqp-processors/pom.xml  |   2 +-
 .../apache/nifi/amqp/processors/PublishAMQP.java   |  78 ++-
 .../nifi/amqp/processors/PublishAMQPTest.java      | 111 ++--
 .../nifi-asn1-bundle/nifi-asn1-services/pom.xml    |  34 +-
 .../asn1bean/compiler/BerClassWriterFactory.java   |  37 ++
 .../java/org/apache/nifi/jasn1/JASN1Reader.java    |  97 +++-
 .../additionalDetails.html                         |  66 ++-
 .../org/apache/nifi/jasn1/JASN1ReaderTest.java     | 152 +++++-
 .../src/test/resources/cant_compile.asn            |  12 +
 .../test/resources/cant_compile_mac_windows.asn    |  11 +
 .../src/test/resources/cant_parse.asn              |  21 +
 .../aws/wag/AbstractAWSGatewayApiProcessor.java    |   9 +-
 .../aws/wag/TestInvokeAmazonGatewayApiMock.java    |   5 +-
 .../cdc/mysql/processors/CaptureChangeMySQL.java   |  42 +-
 .../mysql/processors/CaptureChangeMySQLTest.groovy |  56 ++-
 .../ElasticSearchClientServiceImpl.java            |  10 +-
 .../AbstractJsonQueryElasticsearch.java            |   8 +-
 .../AbstractPaginatedJsonQueryElasticsearch.java   |   3 +
 ...tractPaginatedJsonQueryElasticsearchTest.groovy |  39 ++
 .../additionalDetails.html                         |   2 +-
 .../additionalDetails.html                         |   2 +-
 .../apache/nifi/processor/util/bin/BinFiles.java   |   4 +-
 .../java/org/apache/nifi/util/db/JdbcCommon.java   |   5 +
 .../nifi/json/AbstractJsonRowRecordReader.java     |   8 +-
 .../nifi/web/NiFiWebApiSecurityConfiguration.java  |   2 +
 .../apache/nifi/web/api/ProcessGroupResource.java  |  11 +-
 .../org/apache/nifi/web/api/dto/DtoFactory.java    |   2 +
 .../security/csrf/SkipReplicatedCsrfFilter.java    |  61 +++
 .../csrf/SkipReplicatedCsrfFilterTest.java         | 113 +++++
 .../nifi-kafka-1-0-processors/pom.xml              |  13 +
 .../kafka/pubsub/ConsumeKafkaRecord_1_0.java       |  50 +-
 .../processors/kafka/pubsub/ConsumeKafka_1_0.java  |  58 ++-
 .../processors/kafka/pubsub/ConsumerLease.java     |  58 +--
 .../kafka/pubsub/KafkaProcessorUtils.java          | 405 ---------------
 .../kafka/pubsub/PublishKafkaRecord_1_0.java       |  72 ++-
 .../processors/kafka/pubsub/PublishKafka_1_0.java  |  63 +--
 .../record/sink/kafka/KafkaRecordSink_1_0.java     | 131 +----
 .../processors/kafka/pubsub/ConsumeKafkaTest.java  |  43 +-
 .../processors/kafka/pubsub/ConsumerPoolTest.java  |  15 +-
 .../processors/kafka/pubsub/ITConsumeKafka.java    |  12 +-
 .../kafka/pubsub/TestConsumeKafkaRecord_1_0.java   |  33 +-
 .../record/sink/kafka/TestKafkaRecordSink_1_0.java |  12 +-
 .../src/test/resources/log4j.properties            |  21 -
 .../src/test/resources/zookeeper.properties        |  20 -
 .../nifi-kafka-2-0-processors/pom.xml              |  14 +-
 .../kafka/pubsub/ConsumeKafkaRecord_2_0.java       |  61 +--
 .../processors/kafka/pubsub/ConsumeKafka_2_0.java  |  65 ++-
 .../processors/kafka/pubsub/ConsumerLease.java     |  41 +-
 .../kafka/pubsub/KafkaProcessorUtils.java          | 557 ---------------------
 .../kafka/pubsub/PublishKafkaRecord_2_0.java       |  83 ++-
 .../processors/kafka/pubsub/PublishKafka_2_0.java  |  73 +--
 .../record/sink/kafka/KafkaRecordSink_2_0.java     | 128 +----
 .../kafka/pubsub/ITConsumeKafka_2_0.java           |  12 +-
 .../kafka/pubsub/TestConsumeKafkaRecord_2_0.java   |  64 ++-
 .../kafka/pubsub/TestConsumeKafka_2_0.java         |  43 +-
 .../kafka/pubsub/TestConsumerPartitionsUtil.java   |   2 +-
 .../kafka/pubsub/TestPublishKafkaRecord_2_0.java   |   9 +-
 .../kafka/pubsub/TestPublishKafka_2_0.java         |   7 +-
 .../record/sink/kafka/TestKafkaRecordSink_2_0.java |  14 +-
 .../src/test/resources/log4j.properties            |  21 -
 .../processors/kafka/pubsub/PublisherLease.java    |   3 +-
 .../additionalDetails.html                         |   2 +-
 .../pubsub/TestPublishKafkaMockParameterized.java  |  97 ++--
 ...> TestPublishKafkaRecordMockParameterized.java} |  49 +-
 .../parameterized/flowfileInput1.json              |   0
 .../parameterized/flowfileInputA.json              |   0
 .../Publish/parameterized/kafkaOutput1A.json       |  18 +
 .../Publish/parameterized/kafkaOutput1B.json       |  18 +
 .../Publish/parameterized/kafkaOutputA1.json       |  22 +
 .../Publish/parameterized/kafkaOutputA2.json       |  22 +
 ...flowfileInput1.json => flowfileInputDoc1V.json} |   0
 .../parameterized/flowfileInputDoc1W.json          |  15 +
 .../parameterized/flowfileInputDoc2W.json          |  15 +
 .../parameterized/kafkaOutputDoc1V.json            |  21 +
 .../parameterized/kafkaOutputDoc1W.json            |  18 +
 .../parameterized/kafkaOutputDoc2W.json            |  16 +
 .../apache/nifi/processors/solr/TestQuerySolr.java |  24 +-
 .../nifi/processors/splunk/TestPutSplunkHTTP.java  |   5 +-
 .../nifi/processors/standard/GenerateFlowFile.java |  22 +-
 .../processors/standard/PutDatabaseRecord.java     |   8 +
 .../nifi/json/TestJsonTreeRowRecordReader.java     |   2 +-
 .../nifi-registry-core/nifi-registry-test/pom.xml  |   2 +-
 83 files changed, 1660 insertions(+), 1926 deletions(-)
 create mode 100644 nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/com/beanit/asn1bean/compiler/BerClassWriterFactory.java
 create mode 100644 nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/resources/cant_compile.asn
 create mode 100644 nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/resources/cant_compile_mac_windows.asn
 create mode 100644 nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/resources/cant_parse.asn
 create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/csrf/SkipReplicatedCsrfFilter.java
 create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/csrf/SkipReplicatedCsrfFilterTest.java
 delete mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
 delete mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/resources/log4j.properties
 delete mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/resources/zookeeper.properties
 delete mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
 delete mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/resources/log4j.properties
 copy nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/{TestPublishKafkaMockParameterized.java => TestPublishKafkaRecordMockParameterized.java} (86%)
 copy nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/{PublishRecord => Publish}/parameterized/flowfileInput1.json (100%)
 copy nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/{PublishRecord => Publish}/parameterized/flowfileInputA.json (100%)
 create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/kafkaOutput1A.json
 create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/kafkaOutput1B.json
 create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/kafkaOutputA1.json
 create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/kafkaOutputA2.json
 copy nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/{flowfileInput1.json => flowfileInputDoc1V.json} (100%)
 create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/flowfileInputDoc1W.json
 create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/flowfileInputDoc2W.json
 create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/kafkaOutputDoc1V.json
 create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/kafkaOutputDoc1W.json
 create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/kafkaOutputDoc2W.json


[nifi] 20/22: NIFI-10910 Upgraded amqp-client from 5.15.0 to 5.16.0

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 8e0eeded009e9899941de740f36a74f71709161e
Author: mr1716 <ms...@rit.edu>
AuthorDate: Wed Nov 30 09:20:34 2022 -0500

    NIFI-10910 Upgraded amqp-client from 5.15.0 to 5.16.0
    
    This closes #6734
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

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 d2ae703818..587029882b 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
@@ -20,7 +20,7 @@ language governing permissions and limitations under the License. -->
     <packaging>jar</packaging>
 
     <properties>
-        <amqp-client.version>5.15.0</amqp-client.version>
+        <amqp-client.version>5.16.0</amqp-client.version>
     </properties>
 
     <dependencies>


[nifi] 18/22: NIFI-10891 Replaced HashMap with LinkedHashMap in BinFiles

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 8c74f54b9d4eb6a41b47094b4eec3effbd62e452
Author: sopan98 <ph...@gmail.com>
AuthorDate: Mon Nov 28 23:57:16 2022 -0600

    NIFI-10891 Replaced HashMap with LinkedHashMap in BinFiles
    
    - Resolved non-deterministic behavior in TestMergeContent.testDefragmentMultipleMingledSegments
    
    This closes #6727
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 .../src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-bin-manager/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java b/nifi-nar-bundles/nifi-extension-utils/nifi-bin-manager/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java
index c9d40829dd..ba363bb58e 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-bin-manager/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-bin-manager/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java
@@ -36,7 +36,7 @@ import org.apache.nifi.processor.util.StandardValidators;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Queue;
@@ -280,7 +280,7 @@ public abstract class BinFiles extends AbstractSessionFactoryProcessor {
                 break;
             }
 
-            final Map<String, List<FlowFile>> flowFileGroups = new HashMap<>();
+            final Map<String, List<FlowFile>> flowFileGroups = new LinkedHashMap<>();
             for (FlowFile flowFile : flowFiles) {
                 flowFile = this.preprocessFlowFile(context, session, flowFile);
 


[nifi] 03/22: NIFI-6428 Add 'inTransaction' value to state,fix bug

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 0a09aa57fd9d69626cc6b2a1ae24fb515f1d4944
Author: sssqhai <sh...@outlook.com>
AuthorDate: Sat May 7 16:35:08 2022 +0800

    NIFI-6428 Add 'inTransaction' value to state,fix bug
    
    Signed-off-by: Matthew Burgess <ma...@apache.org>
    
    This closes #6036
---
 .../cdc/mysql/processors/CaptureChangeMySQL.java   |  26 +-
 .../mysql/processors/CaptureChangeMySQLTest.groovy |  21 +-
 .../nifi/cdc/mysql/CaptureChangeMySQLTest.java     | 393 +++++++++++++++++++++
 .../nifi/cdc/mysql/MockBinlogClientJava.java       | 108 ++++++
 4 files changed, 535 insertions(+), 13 deletions(-)

diff --git a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/main/java/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQL.java b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/main/java/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQL.java
index cc9d86137c..3e245c7ab0 100644
--- a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/main/java/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQL.java
+++ b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/main/java/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQL.java
@@ -130,6 +130,8 @@ import static com.github.shyiko.mysql.binlog.event.EventType.PRE_GA_DELETE_ROWS;
 import static com.github.shyiko.mysql.binlog.event.EventType.PRE_GA_WRITE_ROWS;
 import static com.github.shyiko.mysql.binlog.event.EventType.ROTATE;
 import static com.github.shyiko.mysql.binlog.event.EventType.WRITE_ROWS;
+import static com.github.shyiko.mysql.binlog.event.EventType.XID;
+
 
 /**
  * A processor to retrieve Change Data Capture (CDC) events and send them as flow files.
@@ -626,6 +628,8 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
         } else {
             currentSequenceId.set(Long.parseLong(seqIdString));
         }
+        //get inTransaction value from state
+        inTransaction = "true".equals(stateMap.get("inTransaction"));
 
         // Get reference to Distributed Cache if one exists. If it does not, no enrichment (resolution of column names, e.g.) will be performed
         boolean createEnrichmentConnection = false;
@@ -942,6 +946,8 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
                             currentSequenceId.set(beginEventWriter.writeEvent(currentSession, transitUri, beginEvent, currentSequenceId.get(), REL_SUCCESS));
                         }
                         inTransaction = true;
+                        //update inTransaction value to state
+                        updateState(session);
                     } else if ("COMMIT".equals(sql)) {
                         if (!inTransaction) {
                             throw new IOException("COMMIT event received while not processing a transaction (i.e. no corresponding BEGIN event). "
@@ -954,12 +960,11 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
                                     : new CommitTransactionEventInfo(currentDatabase, timestamp, currentBinlogFile, currentBinlogPosition);
                             currentSequenceId.set(commitEventWriter.writeEvent(currentSession, transitUri, commitTransactionEvent, currentSequenceId.get(), REL_SUCCESS));
                         }
-
+                        //update inTransaction value to state
+                        inTransaction = false;
                         updateState(session);
-
                         // Commit the NiFi session
                         session.commitAsync();
-                        inTransaction = false;
                         currentTable = null;
                     } else {
                         // Check for DDL events (alter table, e.g.). Normalize the query to do string matching on the type of change
@@ -1005,9 +1010,12 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
                         currentSequenceId.set(commitEventWriter.writeEvent(currentSession, transitUri, commitTransactionEvent, currentSequenceId.get(), REL_SUCCESS));
                     }
                     // Commit the NiFi session
+                    // update inTransaction value and save next position
+                    // so when restart this processor,we will not read xid event again
+                    inTransaction = false;
+                    currentBinlogPosition = header.getNextPosition();
                     updateState(session);
                     session.commitAsync();
-                    inTransaction = false;
                     currentTable = null;
                     currentDatabase = null;
                     break;
@@ -1089,7 +1097,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
             // Advance the current binlog position. This way if no more events are received and the processor is stopped, it will resume after the event that was just processed.
             // We always get ROTATE and FORMAT_DESCRIPTION messages no matter where we start (even from the end), and they won't have the correct "next position" value, so only
             // advance the position if it is not that type of event.
-            if (eventType != ROTATE && eventType != FORMAT_DESCRIPTION && !useGtid) {
+            if (eventType != ROTATE && eventType != FORMAT_DESCRIPTION && !useGtid && eventType != XID) {
                 currentBinlogPosition = header.getNextPosition();
             }
         }
@@ -1133,10 +1141,10 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
     }
 
     private void updateState(ProcessSession session) throws IOException {
-        updateState(session, currentBinlogFile, currentBinlogPosition, currentSequenceId.get(), currentGtidSet);
+        updateState(session, currentBinlogFile, currentBinlogPosition, currentSequenceId.get(), currentGtidSet, inTransaction);
     }
 
-    private void updateState(ProcessSession session, String binlogFile, long binlogPosition, long sequenceId, String gtidSet) throws IOException {
+    private void updateState(ProcessSession session, String binlogFile, long binlogPosition, long sequenceId, String gtidSet, boolean inTransaction) throws IOException {
         // Update state with latest values
         final Map<String, String> newStateMap = new HashMap<>(session.getState(Scope.CLUSTER).toMap());
 
@@ -1147,6 +1155,8 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
 
         newStateMap.put(BinlogEventInfo.BINLOG_POSITION_KEY, Long.toString(binlogPosition));
         newStateMap.put(EventWriter.SEQUENCE_ID_KEY, String.valueOf(sequenceId));
+        //add inTransaction value into state
+        newStateMap.put("inTransaction", inTransaction ? "true" : "false");
 
         if (gtidSet != null) {
             newStateMap.put(BinlogEventInfo.BINLOG_GTIDSET_KEY, gtidSet);
@@ -1178,7 +1188,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
     }
 
 
-    BinaryLogClient createBinlogClient(String hostname, int port, String username, String password) {
+    protected BinaryLogClient createBinlogClient(String hostname, int port, String username, String password) {
         return new BinaryLogClient(hostname, port, username, password);
     }
 
diff --git a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/groovy/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQLTest.groovy b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/groovy/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQLTest.groovy
index 5b166599fb..37231e533e 100644
--- a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/groovy/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQLTest.groovy
+++ b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/groovy/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQLTest.groovy
@@ -868,7 +868,12 @@ class CaptureChangeMySQLTest {
         testRunner.setProperty(CaptureChangeMySQL.USERNAME, 'root')
         testRunner.setProperty(CaptureChangeMySQL.PASSWORD, 'password')
         testRunner.setProperty(CaptureChangeMySQL.CONNECT_TIMEOUT, '2 seconds')
-
+        final DistributedMapCacheClientImpl cacheClient = createCacheClient()
+        def clientProperties = [:]
+        clientProperties.put(DistributedMapCacheClientService.HOSTNAME.getName(), 'localhost')
+        testRunner.addControllerService('client', cacheClient, clientProperties)
+        testRunner.setProperty(CaptureChangeMySQL.DIST_CACHE_CLIENT, 'client')
+        testRunner.enableControllerService(cacheClient)
         testRunner.run(1, false, true)
 
         // ROTATE
@@ -907,7 +912,7 @@ class CaptureChangeMySQLTest {
         testRunner.run(1, false, false)
 
         testRunner.stateManager.assertStateEquals(BinlogEventInfo.BINLOG_FILENAME_KEY, 'master.000001', Scope.CLUSTER)
-        testRunner.stateManager.assertStateEquals(BinlogEventInfo.BINLOG_POSITION_KEY, '4', Scope.CLUSTER)
+        testRunner.stateManager.assertStateEquals(BinlogEventInfo.BINLOG_POSITION_KEY, '6', Scope.CLUSTER)
         testRunner.stateManager.assertStateEquals(BinlogEventInfo.BINLOG_GTIDSET_KEY, null, Scope.CLUSTER)
 
         // COMMIT
@@ -931,6 +936,12 @@ class CaptureChangeMySQLTest {
         testRunner.setProperty(CaptureChangeMySQL.PASSWORD, 'password')
         testRunner.setProperty(CaptureChangeMySQL.CONNECT_TIMEOUT, '2 seconds')
         testRunner.setProperty(CaptureChangeMySQL.USE_BINLOG_GTID, 'true')
+        final DistributedMapCacheClientImpl cacheClient = createCacheClient()
+        def clientProperties = [:]
+        clientProperties.put(DistributedMapCacheClientService.HOSTNAME.getName(), 'localhost')
+        testRunner.addControllerService('client', cacheClient, clientProperties)
+        testRunner.setProperty(CaptureChangeMySQL.DIST_CACHE_CLIENT, 'client')
+        testRunner.enableControllerService(cacheClient)
 
         testRunner.run(1, false, true)
 
@@ -956,7 +967,7 @@ class CaptureChangeMySQLTest {
 
         // Stop the processor and verify the state is set
         testRunner.stateManager.assertStateEquals(BinlogEventInfo.BINLOG_FILENAME_KEY, '', Scope.CLUSTER)
-        testRunner.stateManager.assertStateEquals(BinlogEventInfo.BINLOG_POSITION_KEY, '-1000', Scope.CLUSTER)
+        testRunner.stateManager.assertStateEquals(BinlogEventInfo.BINLOG_POSITION_KEY, '6', Scope.CLUSTER)
         testRunner.stateManager.assertStateEquals(BinlogEventInfo.BINLOG_GTIDSET_KEY, 'xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx:1-1', Scope.CLUSTER)
 
         ((CaptureChangeMySQL) testRunner.getProcessor()).clearState()
@@ -989,7 +1000,7 @@ class CaptureChangeMySQLTest {
         testRunner.run(1, false, false)
 
         testRunner.stateManager.assertStateEquals(BinlogEventInfo.BINLOG_FILENAME_KEY, '', Scope.CLUSTER)
-        testRunner.stateManager.assertStateEquals(BinlogEventInfo.BINLOG_POSITION_KEY, '-1000', Scope.CLUSTER)
+        testRunner.stateManager.assertStateEquals(BinlogEventInfo.BINLOG_POSITION_KEY, '12', Scope.CLUSTER)
         testRunner.stateManager.assertStateEquals(BinlogEventInfo.BINLOG_GTIDSET_KEY, 'xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx:2-2', Scope.CLUSTER)
 
         // GTID
@@ -1013,7 +1024,7 @@ class CaptureChangeMySQLTest {
         testRunner.run(1, true, false)
 
         testRunner.stateManager.assertStateEquals(BinlogEventInfo.BINLOG_FILENAME_KEY, '', Scope.CLUSTER)
-        testRunner.stateManager.assertStateEquals(BinlogEventInfo.BINLOG_POSITION_KEY, '-1000', Scope.CLUSTER)
+        testRunner.stateManager.assertStateEquals(BinlogEventInfo.BINLOG_POSITION_KEY, '18', Scope.CLUSTER)
         testRunner.stateManager.assertStateEquals(BinlogEventInfo.BINLOG_GTIDSET_KEY, 'xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx:2-3', Scope.CLUSTER)
     }
 
diff --git a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/java/org/apache/nifi/cdc/mysql/CaptureChangeMySQLTest.java b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/java/org/apache/nifi/cdc/mysql/CaptureChangeMySQLTest.java
new file mode 100644
index 0000000000..0d3fd0f257
--- /dev/null
+++ b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/java/org/apache/nifi/cdc/mysql/CaptureChangeMySQLTest.java
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.cdc.mysql;
+
+
+import com.github.shyiko.mysql.binlog.BinaryLogClient;
+import com.github.shyiko.mysql.binlog.event.Event;
+import com.github.shyiko.mysql.binlog.event.EventData;
+import com.github.shyiko.mysql.binlog.event.EventHeaderV4;
+import com.github.shyiko.mysql.binlog.event.EventType;
+import com.github.shyiko.mysql.binlog.event.QueryEventData;
+import com.github.shyiko.mysql.binlog.event.RotateEventData;
+import com.github.shyiko.mysql.binlog.network.SSLMode;
+import org.apache.commons.io.output.WriterOutputStream;
+import org.apache.nifi.cdc.event.ColumnDefinition;
+import org.apache.nifi.cdc.event.TableInfo;
+import org.apache.nifi.cdc.event.TableInfoCacheKey;
+import org.apache.nifi.cdc.mysql.processors.CaptureChangeMySQL;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.distributed.cache.client.Deserializer;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.state.MockStateManager;
+import org.apache.nifi.util.MockComponentLog;
+import org.apache.nifi.util.MockControllerServiceInitializationContext;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeoutException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class CaptureChangeMySQLTest {
+
+    private static final String DRIVER_LOCATION = "http://mysql-driver.com/driver.jar";
+    CaptureChangeMySQL processor;
+    TestRunner testRunner;
+    MockBinlogClientJava client = new MockBinlogClientJava("localhost", 3306, "root", "password");
+
+    @BeforeEach
+    void setUp() throws Exception {
+        processor = new MockCaptureChangeMySQL();
+        testRunner = TestRunners.newTestRunner(processor);
+    }
+
+    @Test
+    void testConnectionFailures() throws Exception {
+        testRunner.setProperty(CaptureChangeMySQL.DRIVER_LOCATION, DRIVER_LOCATION);
+        testRunner.setProperty(CaptureChangeMySQL.HOSTS, "localhost:3306");
+        testRunner.setProperty(CaptureChangeMySQL.USERNAME, "root");
+        testRunner.setProperty(CaptureChangeMySQL.SERVER_ID, "1");
+        final DistributedMapCacheClientImpl cacheClient = createCacheClient();
+        Map<String, String> clientProperties = new HashMap<>();
+        clientProperties.put(DistributedMapCacheClientService.HOSTNAME.getName(), "localhost");
+        testRunner.addControllerService("client", cacheClient, clientProperties);
+        testRunner.setProperty(CaptureChangeMySQL.DIST_CACHE_CLIENT, "client");
+        testRunner.enableControllerService(cacheClient);
+        client.connectionError = true;
+        try {
+            testRunner.run();
+        } catch (AssertionError ae) {
+            Throwable pe = ae.getCause();
+            assertTrue(pe instanceof ProcessException);
+            Throwable ioe = pe.getCause();
+            assertTrue(ioe instanceof IOException);
+            assertEquals("Could not connect binlog client to any of the specified hosts due to: Error during connect", ioe.getMessage());
+            assertTrue(ioe.getCause() instanceof IOException);
+        }
+        client.connectionError = false;
+
+        client.connectionTimeout = true;
+        try {
+            testRunner.run();
+        } catch (AssertionError ae) {
+            Throwable pe = ae.getCause();
+            assertTrue(pe instanceof ProcessException);
+            Throwable ioe = pe.getCause();
+            assertTrue(ioe instanceof IOException);
+            assertEquals("Could not connect binlog client to any of the specified hosts due to: Connection timed out", ioe.getMessage());
+            assertTrue(ioe.getCause() instanceof TimeoutException);
+        }
+        client.connectionTimeout = false;
+    }
+
+    @Test
+    void testSslModeDisabledSslContextServiceNotRequired() {
+        testRunner.setProperty(CaptureChangeMySQL.HOSTS, "localhost:3306");
+        testRunner.setProperty(CaptureChangeMySQL.SSL_MODE, SSLMode.DISABLED.toString());
+        testRunner.assertValid();
+    }
+
+    @Test
+    void testGetXIDEvents() throws Exception {
+        testRunner.setProperty(CaptureChangeMySQL.DRIVER_LOCATION, DRIVER_LOCATION);
+        testRunner.setProperty(CaptureChangeMySQL.HOSTS, "localhost:3306");
+        testRunner.setProperty(CaptureChangeMySQL.USERNAME, "root");
+        testRunner.setProperty(CaptureChangeMySQL.CONNECT_TIMEOUT, "2 seconds");
+        testRunner.setProperty(CaptureChangeMySQL.INCLUDE_BEGIN_COMMIT, "true");
+        final DistributedMapCacheClientImpl cacheClient = createCacheClient();
+        Map<String, String> clientProperties = new HashMap<>();
+        clientProperties.put(DistributedMapCacheClientService.HOSTNAME.getName(), "localhost");
+        testRunner.addControllerService("client", cacheClient, clientProperties);
+        testRunner.setProperty(CaptureChangeMySQL.DIST_CACHE_CLIENT, "client");
+        testRunner.enableControllerService(cacheClient);
+
+        testRunner.run(1, false, true);
+        // COMMIT
+        EventHeaderV4 header2 = new EventHeaderV4();
+        header2.setEventType(EventType.XID);
+        header2.setNextPosition(12);
+        header2.setTimestamp(new Date().getTime());
+        EventData eventData = new EventData() {
+        };
+        client.sendEvent(new Event(header2, eventData));
+
+        // when we ge a xid event without having got a 'begin' event ,throw an exception
+        assertThrows(AssertionError.class, () -> testRunner.run(1, false, false));
+    }
+
+    @Test
+    void testBeginCommitTransaction() throws Exception {
+        testRunner.setProperty(CaptureChangeMySQL.DRIVER_LOCATION, DRIVER_LOCATION);
+        testRunner.setProperty(CaptureChangeMySQL.HOSTS, "localhost:3306");
+        testRunner.setProperty(CaptureChangeMySQL.USERNAME, "root");
+        testRunner.setProperty(CaptureChangeMySQL.CONNECT_TIMEOUT, "2 seconds");
+        testRunner.setProperty(CaptureChangeMySQL.INCLUDE_BEGIN_COMMIT, "true");
+        final DistributedMapCacheClientImpl cacheClient = createCacheClient();
+        Map<String, String> clientProperties = new HashMap<>();
+        clientProperties.put(DistributedMapCacheClientService.HOSTNAME.getName(), "localhost");
+        testRunner.addControllerService("client", cacheClient, clientProperties);
+        testRunner.setProperty(CaptureChangeMySQL.DIST_CACHE_CLIENT, "client");
+        testRunner.enableControllerService(cacheClient);
+
+
+        testRunner.run(1, false, true);
+
+        EventHeaderV4 header = new EventHeaderV4();
+        header.setEventType(EventType.ROTATE);
+        header.setNextPosition(2);
+        header.setTimestamp(new Date().getTime());
+        RotateEventData rotateEventData = new RotateEventData();
+        rotateEventData.setBinlogFilename("mysql-bin.000001");
+        rotateEventData.setBinlogPosition(4L);
+        client.sendEvent(new Event(header, rotateEventData));
+
+        // BEGIN
+        EventHeaderV4 header1 = new EventHeaderV4();
+        header1.setEventType(EventType.QUERY);
+        header1.setNextPosition(6);
+        header1.setTimestamp(new Date().getTime());
+        QueryEventData rotateEventData1 = new QueryEventData();
+        rotateEventData1.setDatabase("mysql-bin.000001");
+        rotateEventData1.setDatabase("myDB");
+        rotateEventData1.setSql("BEGIN");
+        client.sendEvent(new Event(header1, rotateEventData1));
+
+        // COMMIT
+        EventHeaderV4 header2 = new EventHeaderV4();
+        header2.setEventType(EventType.XID);
+        header2.setNextPosition(12);
+        header2.setTimestamp(new Date().getTime());
+        EventData eventData2 = new EventData() {
+        };
+        client.sendEvent(new Event(header2, eventData2));
+
+        //when get a xid event,stop and restart the processor
+        //here we used to get an exception
+        testRunner.run(1, true, false);
+        testRunner.run(1, false, false);
+
+        // next transaction
+        // BEGIN
+        EventHeaderV4 header3 = new EventHeaderV4();
+        header3.setEventType(EventType.QUERY);
+        header3.setNextPosition(16);
+        header3.setTimestamp(new Date().getTime());
+        QueryEventData rotateEventData3 = new QueryEventData();
+        rotateEventData3.setDatabase("mysql-bin.000001");
+        rotateEventData3.setDatabase("myDB");
+        rotateEventData3.setSql("BEGIN");
+        client.sendEvent(new Event(header3, rotateEventData3));
+
+
+        testRunner.run(1, true, false);
+    }
+
+    /********************************
+     * Mock and helper classes below
+     ********************************/
+
+    static DistributedMapCacheClientImpl createCacheClient() throws InitializationException {
+
+        final DistributedMapCacheClientImpl client = new DistributedMapCacheClientImpl();
+        final ComponentLog logger = new MockComponentLog("client", client);
+        final MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client", logger, new MockStateManager(client));
+
+        client.initialize(clientInitContext);
+
+        return client;
+    }
+
+    static final class DistributedMapCacheClientImpl extends AbstractControllerService implements DistributedMapCacheClient {
+
+        private final Map<String, String> cacheMap = new HashMap<>();
+
+
+        @Override
+        protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+            List<PropertyDescriptor> descriptors = new ArrayList<>();
+            descriptors.add(DistributedMapCacheClientService.HOSTNAME);
+            descriptors.add(DistributedMapCacheClientService.COMMUNICATIONS_TIMEOUT);
+            descriptors.add(DistributedMapCacheClientService.PORT);
+            descriptors.add(DistributedMapCacheClientService.SSL_CONTEXT_SERVICE);
+            return descriptors;
+        }
+
+        @Override
+        public <K, V> boolean putIfAbsent(
+                final K key,
+                final V value,
+                final Serializer<K> keySerializer, final Serializer<V> valueSerializer) throws IOException {
+
+            StringWriter keyWriter = new StringWriter();
+            keySerializer.serialize(key, new WriterOutputStream(keyWriter));
+            String keyString = keyWriter.toString();
+
+            if (cacheMap.containsKey(keyString)) return false;
+
+            StringWriter valueWriter = new StringWriter();
+            valueSerializer.serialize(value, new WriterOutputStream(valueWriter));
+            return true;
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public <K, V> V getAndPutIfAbsent(
+                final K key, final V value, final Serializer<K> keySerializer, final Serializer<V> valueSerializer,
+                final Deserializer<V> valueDeserializer) throws IOException {
+            StringWriter keyWriter = new StringWriter();
+            keySerializer.serialize(key, new WriterOutputStream(keyWriter));
+            String keyString = keyWriter.toString();
+
+            if (cacheMap.containsKey(keyString))
+                return valueDeserializer.deserialize(cacheMap.get(keyString).getBytes(StandardCharsets.UTF_8));
+
+            StringWriter valueWriter = new StringWriter();
+            valueSerializer.serialize(value, new WriterOutputStream(valueWriter));
+            return null;
+        }
+
+        @Override
+        public <K> boolean containsKey(final K key, final Serializer<K> keySerializer) throws IOException {
+            StringWriter keyWriter = new StringWriter();
+            keySerializer.serialize(key, new WriterOutputStream(keyWriter));
+            String keyString = keyWriter.toString();
+
+            return cacheMap.containsKey(keyString);
+        }
+
+        @Override
+        public <K, V> V get(
+                final K key,
+                final Serializer<K> keySerializer, final Deserializer<V> valueDeserializer) throws IOException {
+            StringWriter keyWriter = new StringWriter();
+            keySerializer.serialize(key, new WriterOutputStream(keyWriter));
+            String keyString = keyWriter.toString();
+
+            return (cacheMap.containsKey(keyString)) ? valueDeserializer.deserialize(cacheMap.get(keyString).getBytes(StandardCharsets.UTF_8)) : null;
+        }
+
+        @Override
+        public void close() throws IOException {
+
+        }
+
+        @Override
+        public <K> boolean remove(final K key, final Serializer<K> serializer) throws IOException {
+            StringWriter keyWriter = new StringWriter();
+            serializer.serialize(key, new WriterOutputStream(keyWriter));
+            String keyString = keyWriter.toString();
+
+            boolean removed = (cacheMap.containsKey(keyString));
+            cacheMap.remove(keyString);
+            return removed;
+        }
+
+        @Override
+        public long removeByPattern(String regex) throws IOException {
+            final List<String> removedRecords = new ArrayList<>();
+            Pattern p = Pattern.compile(regex);
+            for (String key : cacheMap.keySet()) {
+                // Key must be backed by something that can be converted into a String
+                Matcher m = p.matcher(key);
+                if (m.matches()) {
+                    removedRecords.add(cacheMap.get(key));
+                }
+            }
+            final long numRemoved = removedRecords.size();
+            for (String it : removedRecords) {
+                cacheMap.remove(it);
+            }
+            return numRemoved;
+        }
+
+        @Override
+        public <K, V> void put(
+                final K key,
+                final V value,
+                final Serializer<K> keySerializer, final Serializer<V> valueSerializer) throws IOException {
+            StringWriter keyWriter = new StringWriter();
+            keySerializer.serialize(key, new WriterOutputStream(keyWriter));
+            StringWriter valueWriter = new StringWriter();
+            valueSerializer.serialize(value, new WriterOutputStream(valueWriter));
+        }
+    }
+
+    public class MockCaptureChangeMySQL extends CaptureChangeMySQL {
+
+        Map<TableInfoCacheKey, TableInfo> cache = new HashMap<>();
+
+        public BinaryLogClient createBinlogClient(String hostname, int port, String username, String password) {
+            return client;
+        }
+
+        @Override
+        public TableInfo loadTableInfo(TableInfoCacheKey key) {
+            TableInfo tableInfo = cache.get(key);
+            if (tableInfo == null) {
+                List<ColumnDefinition> column = new ArrayList<>();
+                column.add(new ColumnDefinition((byte) 4, "id"));
+                column.add(new ColumnDefinition((byte) -4, "string1"));
+
+                tableInfo = new TableInfo(key.getDatabaseName(), key.getTableName(), key.getTableId(), column);
+                cache.put(key, tableInfo);
+            }
+            return tableInfo;
+        }
+
+        @Override
+        protected void registerDriver(String locationString, String drvName) throws InitializationException {
+        }
+
+        @Override
+        protected Connection getJdbcConnection() throws SQLException {
+            Connection mockConnection = mock(Connection.class);
+            Statement mockStatement = mock(Statement.class);
+            when(mockConnection.createStatement()).thenReturn(mockStatement);
+            ResultSet mockResultSet = mock(ResultSet.class);
+            when(mockStatement.executeQuery(anyString())).thenReturn(mockResultSet);
+            return mockConnection;
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/java/org/apache/nifi/cdc/mysql/MockBinlogClientJava.java b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/java/org/apache/nifi/cdc/mysql/MockBinlogClientJava.java
new file mode 100644
index 0000000000..d23822292e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/java/org/apache/nifi/cdc/mysql/MockBinlogClientJava.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.cdc.mysql;
+
+import com.github.shyiko.mysql.binlog.BinaryLogClient;
+import com.github.shyiko.mysql.binlog.event.Event;
+import com.github.shyiko.mysql.binlog.network.SSLSocketFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+
+public class MockBinlogClientJava extends BinaryLogClient {
+    String hostname;
+    int port;
+    String username;
+    String password;
+
+    boolean connected;
+    public boolean connectionTimeout = false;
+    public boolean connectionError = false;
+
+    List<LifecycleListener> lifecycleListeners = new ArrayList<>();
+    SSLSocketFactory sslSocketFactory;
+
+    List<EventListener> eventListeners = new ArrayList<>();
+
+
+    public MockBinlogClientJava(String hostname, int port, String username, String password) {
+        super(hostname, port, username, password);
+        this.hostname = hostname;
+        this.port = port;
+        this.username = username;
+        this.password = password;
+    }
+
+    @Override
+    public void connect(long timeoutInMilliseconds) throws IOException, TimeoutException {
+        if (connectionTimeout) {
+            throw new TimeoutException("Connection timed out");
+        }
+        if (connectionError) {
+            throw new IOException("Error during connect");
+        }
+        if (password == null) {
+            throw new NullPointerException("Password can't be null");
+        }
+        connected = true;
+    }
+
+    @Override
+    public void disconnect() throws IOException {
+        connected = false;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return connected;
+    }
+
+    @Override
+    public void registerEventListener(EventListener eventListener) {
+        eventListeners.add(eventListener);
+    }
+
+    public void unregisterEventListener(EventListener eventListener) {
+        eventListeners.remove(eventListener);
+    }
+
+    @Override
+    public void registerLifecycleListener(LifecycleListener lifecycleListener) {
+        if (!lifecycleListeners.contains(lifecycleListener)) {
+            lifecycleListeners.add(lifecycleListener);
+        }
+    }
+
+    @Override
+    public void unregisterLifecycleListener(LifecycleListener lifecycleListener) {
+        lifecycleListeners.remove(lifecycleListener);
+    }
+
+    @Override
+    public void setSslSocketFactory(SSLSocketFactory sslSocketFactory) {
+        super.setSslSocketFactory(sslSocketFactory);
+        this.sslSocketFactory = sslSocketFactory;
+    }
+
+    public void sendEvent(Event event) {
+        for (EventListener eventListener : eventListeners) {
+            eventListener.onEvent(event);
+        }
+    }
+}


[nifi] 16/22: NIFI-10785 Allow publishing AMQP message with null header value NIFI-10785 addressing review comment NIFI-10785 addressing review comments (remove unnecessary property to ignore null headers)

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 71f41b96baf602859c2d1ed9383ff1fd13268719
Author: Nandor Soma Abonyi <ab...@gmail.com>
AuthorDate: Thu Nov 10 23:45:52 2022 +0100

    NIFI-10785 Allow publishing AMQP message with null header value
    NIFI-10785 addressing review comment
    NIFI-10785 addressing review comments (remove unnecessary property to ignore null headers)
    
    Signed-off-by: Nathan Gough <th...@gmail.com>
    
    This closes #6649.
---
 .../apache/nifi/amqp/processors/PublishAMQP.java   |  78 +++++++--------
 .../nifi/amqp/processors/PublishAMQPTest.java      | 111 +++++++++++++--------
 2 files changed, 103 insertions(+), 86 deletions(-)

diff --git a/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/PublishAMQP.java b/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/PublishAMQP.java
index 4bd94f3a41..6a8c7ac645 100644
--- a/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/PublishAMQP.java
+++ b/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/PublishAMQP.java
@@ -16,25 +16,15 @@
  */
 package org.apache.nifi.amqp.processors;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.function.Consumer;
-import java.util.regex.Pattern;
-
-import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import com.rabbitmq.client.AMQP;
+import com.rabbitmq.client.AMQP.BasicProperties;
+import com.rabbitmq.client.Connection;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
-import org.apache.nifi.annotation.behavior.SystemResource;
 import org.apache.nifi.annotation.behavior.ReadsAttribute;
 import org.apache.nifi.annotation.behavior.ReadsAttributes;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
@@ -45,13 +35,19 @@ import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.stream.io.StreamUtils;
 
-import com.rabbitmq.client.AMQP;
-import com.rabbitmq.client.AMQP.BasicProperties;
-import com.rabbitmq.client.Connection;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.regex.Pattern;
 
 @Tags({ "amqp", "rabbit", "put", "message", "send", "publish" })
 @InputRequirement(Requirement.INPUT_REQUIRED)
@@ -89,6 +85,7 @@ public class PublishAMQP extends AbstractAMQPProcessor<AMQPPublisher> {
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(Validator.VALID)
             .build();
+
     public static final PropertyDescriptor ROUTING_KEY = new PropertyDescriptor.Builder()
             .name("Routing Key")
             .description("The name of the Routing Key that will be used by AMQP to route messages from the exchange to a destination queue(s). "
@@ -99,6 +96,7 @@ public class PublishAMQP extends AbstractAMQPProcessor<AMQPPublisher> {
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
+
     public static final PropertyDescriptor HEADER_SEPARATOR = new PropertyDescriptor.Builder()
             .name("header.separator")
             .displayName("Header Separator")
@@ -108,10 +106,12 @@ public class PublishAMQP extends AbstractAMQPProcessor<AMQPPublisher> {
             .addValidator(StandardValidators.SINGLE_CHAR_VALIDATOR)
             .required(false)
             .build();
+
     public static final Relationship REL_SUCCESS = new Relationship.Builder()
             .name("success")
             .description("All FlowFiles that are sent to the AMQP destination are routed to this relationship")
             .build();
+
     public static final Relationship REL_FAILURE = new Relationship.Builder()
             .name("failure")
             .description("All FlowFiles that cannot be routed to the AMQP destination are routed to this relationship")
@@ -144,7 +144,7 @@ public class PublishAMQP extends AbstractAMQPProcessor<AMQPPublisher> {
      *
      * NOTE: Attributes extracted from {@link FlowFile} are considered
      * candidates for AMQP properties if their names are prefixed with
-     * {@link AMQPUtils#AMQP_PROP_PREFIX} (e.g., amqp$contentType=text/xml)
+     * {@link PublishAMQP#ATTRIBUTES_PREFIX} (e.g., amqp$contentType=text/xml)
      */
     @Override
     protected void processResource(final Connection connection, final AMQPPublisher publisher, ProcessContext context, ProcessSession session) throws ProcessException {
@@ -153,14 +153,16 @@ public class PublishAMQP extends AbstractAMQPProcessor<AMQPPublisher> {
             return;
         }
 
-        final BasicProperties amqpProperties = extractAmqpPropertiesFromFlowFile(flowFile,
-                context.getProperty(HEADER_SEPARATOR).toString().charAt(0));
         final String routingKey = context.getProperty(ROUTING_KEY).evaluateAttributeExpressions(flowFile).getValue();
         if (routingKey == null) {
             throw new IllegalArgumentException("Failed to determine 'routing key' with provided value '"
                 + context.getProperty(ROUTING_KEY) + "' after evaluating it as expression against incoming FlowFile.");
         }
 
+        final Character separator = context.getProperty(HEADER_SEPARATOR).toString().charAt(0);
+
+        final BasicProperties amqpProperties = extractAmqpPropertiesFromFlowFile(flowFile, separator);
+
         final String exchange = context.getProperty(EXCHANGE).evaluateAttributeExpressions(flowFile).getValue();
         final byte[] messageContent = extractMessage(flowFile, session);
 
@@ -199,12 +201,7 @@ public class PublishAMQP extends AbstractAMQPProcessor<AMQPPublisher> {
      */
     private byte[] extractMessage(FlowFile flowFile, ProcessSession session){
         final byte[] messageContent = new byte[(int) flowFile.getSize()];
-        session.read(flowFile, new InputStreamCallback() {
-            @Override
-            public void process(final InputStream in) throws IOException {
-                StreamUtils.fillBuffer(in, messageContent, true);
-            }
-        });
+        session.read(flowFile, in -> StreamUtils.fillBuffer(in, messageContent, true));
         return messageContent;
     }
 
@@ -226,16 +223,9 @@ public class PublishAMQP extends AbstractAMQPProcessor<AMQPPublisher> {
      * Extracts AMQP properties from the {@link FlowFile} attributes. Attributes
      * extracted from {@link FlowFile} are considered candidates for AMQP
      * properties if their names are prefixed with
-     * {@link AMQPUtils#AMQP_PROP_PREFIX} (e.g., amqp$contentType=text/xml).
-     *
-     * Some fields require a specific format and are validated:
-     *
-     * {@link AMQPUtils#validateAMQPHeaderProperty}
-     * {@link AMQPUtils#validateAMQPDeliveryModeProperty}
-     * {@link AMQPUtils#validateAMQPPriorityProperty}
-     * {@link AMQPUtils#validateAMQPTimestampProperty}
+     * {@link PublishAMQP#ATTRIBUTES_PREFIX} (e.g., amqp$contentType=text/xml).
      */
-    private BasicProperties extractAmqpPropertiesFromFlowFile(FlowFile flowFile,Character headerSeparator) {
+    private BasicProperties extractAmqpPropertiesFromFlowFile(FlowFile flowFile, Character headerSeparator) {
         final AMQP.BasicProperties.Builder builder = new AMQP.BasicProperties.Builder();
 
         updateBuilderFromAttribute(flowFile, "contentType", builder::contentType);
@@ -251,7 +241,7 @@ public class PublishAMQP extends AbstractAMQPProcessor<AMQPPublisher> {
         updateBuilderFromAttribute(flowFile, "userId", builder::userId);
         updateBuilderFromAttribute(flowFile, "appId", builder::appId);
         updateBuilderFromAttribute(flowFile, "clusterId", builder::clusterId);
-        updateBuilderFromAttribute(flowFile, "headers", headers -> builder.headers(validateAMQPHeaderProperty(headers,headerSeparator)));
+        updateBuilderFromAttribute(flowFile, "headers", headers -> builder.headers(validateAMQPHeaderProperty(headers, headerSeparator)));
 
         return builder.build();
     }
@@ -263,15 +253,17 @@ public class PublishAMQP extends AbstractAMQPProcessor<AMQPPublisher> {
      * @param amqpPropValue the value of the property
      * @return {@link Map} if valid otherwise null
      */
-    private Map<String, Object> validateAMQPHeaderProperty(String amqpPropValue,Character splitValue) {
-        String[] strEntries = amqpPropValue.split(Pattern.quote(String.valueOf(splitValue)));
-        Map<String, Object> headers = new HashMap<>();
+    private Map<String, Object> validateAMQPHeaderProperty(String amqpPropValue, Character splitValue) {
+        final String[] strEntries = amqpPropValue.split(Pattern.quote(String.valueOf(splitValue)));
+        final Map<String, Object> headers = new HashMap<>();
         for (String strEntry : strEntries) {
-            String[] kv = strEntry.split("=");
+            final String[] kv = strEntry.split("=", -1); // without using limit, trailing delimiter would be ignored
             if (kv.length == 2) {
                 headers.put(kv[0].trim(), kv[1].trim());
+            } else if (kv.length == 1) {
+                headers.put(kv[0].trim(), null);
             } else {
-                getLogger().warn("Malformed key value pair for AMQP header property: " + amqpPropValue);
+                getLogger().warn(String.format("Malformed key value pair in AMQP header property (%s): %s", amqpPropValue, strEntry));
             }
         }
         return headers;
diff --git a/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/test/java/org/apache/nifi/amqp/processors/PublishAMQPTest.java b/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/test/java/org/apache/nifi/amqp/processors/PublishAMQPTest.java
index 664862576f..72866ef4f6 100644
--- a/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/test/java/org/apache/nifi/amqp/processors/PublishAMQPTest.java
+++ b/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/test/java/org/apache/nifi/amqp/processors/PublishAMQPTest.java
@@ -16,6 +16,15 @@
  */
 package org.apache.nifi.amqp.processors;
 
+import com.rabbitmq.client.Channel;
+import com.rabbitmq.client.Connection;
+import com.rabbitmq.client.GetResponse;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.Test;
+
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Date;
@@ -24,19 +33,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ExecutorService;
 
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.util.MockFlowFile;
-import org.apache.nifi.util.TestRunner;
-import org.apache.nifi.util.TestRunners;
-
-import com.rabbitmq.client.Channel;
-import com.rabbitmq.client.Connection;
-import com.rabbitmq.client.GetResponse;
-import org.junit.jupiter.api.Test;
-
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class PublishAMQPTest {
@@ -45,11 +43,13 @@ public class PublishAMQPTest {
     public void validateSuccessfulPublishAndTransferToSuccess() throws Exception {
         final PublishAMQP pubProc = new LocalPublishAMQP();
         final TestRunner runner = TestRunners.newTestRunner(pubProc);
-        runner.setProperty(PublishAMQP.BROKERS, "injvm:5672");
-        runner.setProperty(PublishAMQP.EXCHANGE, "myExchange");
-        runner.setProperty(PublishAMQP.ROUTING_KEY, "key1");
-        runner.setProperty(PublishAMQP.USER, "user");
-        runner.setProperty(PublishAMQP.PASSWORD, "password");
+        setConnectionProperties(runner);
+
+        final Map<String, String> expectedHeaders = new HashMap<String, String>() {{
+            put("foo", "bar");
+            put("foo2", "bar2");
+            put("foo3", null);
+        }};
 
         final Map<String, String> attributes = new HashMap<>();
         attributes.put("foo", "bar");
@@ -83,13 +83,7 @@ public class PublishAMQPTest {
 
         final Map<String, Object> headerMap = msg1.getProps().getHeaders();
 
-        final Object foo = headerMap.get("foo");
-        final Object foo2 = headerMap.get("foo2");
-        final Object foo3 = headerMap.get("foo3");
-
-        assertEquals("bar", foo.toString());
-        assertEquals("bar2", foo2.toString());
-        assertNull(foo3);
+        assertEquals(expectedHeaders, headerMap);
 
         assertEquals((Integer) 1, msg1.getProps().getDeliveryMode());
         assertEquals((Integer) 2, msg1.getProps().getPriority());
@@ -110,15 +104,16 @@ public class PublishAMQPTest {
     public void validateSuccessWithHeaderWithCommaPublishToSuccess() throws Exception {
         final PublishAMQP pubProc = new LocalPublishAMQP();
         final TestRunner runner = TestRunners.newTestRunner(pubProc);
-        runner.setProperty(PublishAMQP.BROKERS, "injvm:5672");
-        runner.setProperty(PublishAMQP.EXCHANGE, "myExchange");
-        runner.setProperty(PublishAMQP.ROUTING_KEY, "key1");
-        runner.setProperty(PublishAMQP.USER, "user");
-        runner.setProperty(PublishAMQP.PASSWORD, "password");
+        setConnectionProperties(runner);
         runner.setProperty(PublishAMQP.HEADER_SEPARATOR,"|");
 
-        final Map<String, String> attributes = new HashMap<>();
+        final Map<String, String> expectedHeaders = new HashMap<String, String>() {{
+            put("foo", "(bar,bar)");
+            put("foo2", "bar2");
+            put("foo3", null);
+        }};
 
+        final Map<String, String> attributes = new HashMap<>();
         attributes.put("amqp$headers", "foo=(bar,bar)|foo2=bar2|foo3");
 
         runner.enqueue("Hello Joe".getBytes(), attributes);
@@ -134,14 +129,7 @@ public class PublishAMQPTest {
 
         final Map<String, Object> headerMap = msg1.getProps().getHeaders();
 
-        final Object foo = headerMap.get("foo");
-        final Object foo2 = headerMap.get("foo2");
-        final Object foo3 = headerMap.get("foo3");
-
-        assertEquals("(bar,bar)", foo.toString());
-        assertEquals("bar2", foo2.toString());
-        assertNull(foo3);
-
+        assertEquals(expectedHeaders, headerMap);
 
         assertNotNull(channel.basicGet("queue2", true));
     }
@@ -152,18 +140,48 @@ public class PublishAMQPTest {
         final TestRunner runner = TestRunners.newTestRunner(pubProc);
         runner.setProperty(PublishAMQP.HEADER_SEPARATOR,"|,");
         runner.assertNotValid();
+    }
+
+    @Test
+    public void validateMalformedHeaderIgnoredAndPublishToSuccess() throws Exception {
+        final PublishAMQP pubProc = new LocalPublishAMQP();
+        final TestRunner runner = TestRunners.newTestRunner(pubProc);
+        setConnectionProperties(runner);
+        runner.setProperty(PublishAMQP.HEADER_SEPARATOR,"|");
+
+        final Map<String, String> expectedHeaders = new HashMap<String, String>() {{
+            put("foo", "(bar,bar)");
+            put("foo2", "bar2");
+            put("foo3", null);
+        }};
+
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("amqp$headers", "foo=(bar,bar)|foo2=bar2|foo3|foo4=malformed=|foo5=mal=formed");
+
+        runner.enqueue("Hello Joe".getBytes(), attributes);
+
+        runner.run();
+
+        final MockFlowFile successFF = runner.getFlowFilesForRelationship(PublishAMQP.REL_SUCCESS).get(0);
+        assertNotNull(successFF);
+
+        final Channel channel = ((LocalPublishAMQP) pubProc).getConnection().createChannel();
+        final GetResponse msg1 = channel.basicGet("queue1", true);
+        assertNotNull(msg1);
+
+        final Map<String, Object> headerMap = msg1.getProps().getHeaders();
 
+        assertEquals(expectedHeaders, headerMap);
+
+        assertNotNull(channel.basicGet("queue2", true));
     }
 
     @Test
-    public void validateFailedPublishAndTransferToFailure() throws Exception {
+    public void validateFailedPublishAndTransferToFailure() {
         PublishAMQP pubProc = new LocalPublishAMQP();
         TestRunner runner = TestRunners.newTestRunner(pubProc);
-        runner.setProperty(PublishAMQP.BROKERS, "injvm:5672");
-        runner.setProperty(PublishAMQP.EXCHANGE, "badToTheBone");
-        runner.setProperty(PublishAMQP.ROUTING_KEY, "key1");
-        runner.setProperty(PublishAMQP.USER, "user");
-        runner.setProperty(PublishAMQP.PASSWORD, "password");
+        setConnectionProperties(runner);
+        runner.setProperty(PublishAMQP.EXCHANGE, "nonExistentExchange");
 
         runner.enqueue("Hello Joe".getBytes());
 
@@ -173,6 +191,13 @@ public class PublishAMQPTest {
         assertNotNull(runner.getFlowFilesForRelationship(PublishAMQP.REL_FAILURE).get(0));
     }
 
+    private void setConnectionProperties(TestRunner runner) {
+        runner.setProperty(PublishAMQP.BROKERS, "injvm:5672");
+        runner.setProperty(PublishAMQP.USER, "user");
+        runner.setProperty(PublishAMQP.PASSWORD, "password");
+        runner.setProperty(PublishAMQP.EXCHANGE, "myExchange");
+        runner.setProperty(PublishAMQP.ROUTING_KEY, "key1");
+    }
 
     public static class LocalPublishAMQP extends PublishAMQP {
         private TestConnection connection;


[nifi] 22/22: NIFI-10913 updating versions post commit landing

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit eb0d07fda3f41512a79339d3fa53f3eb4e98d0b1
Author: Joe Witt <jo...@apache.org>
AuthorDate: Wed Nov 30 12:53:22 2022 -0700

    NIFI-10913 updating versions post commit landing
---
 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/pom.xml | 2 +-
 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/pom.xml | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

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 4711fcf04a..6d7676980a 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
@@ -63,7 +63,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-kerberos</artifactId>
-            <version>1.19.0-SNAPSHOT</version>
+            <version>1.19.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
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 5fd2dc2381..76c7c38667 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
@@ -59,7 +59,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-kerberos</artifactId>
-            <version>1.19.0-SNAPSHOT</version>
+            <version>1.19.1-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>


[nifi] 01/22: NIFI-10866 Refactored Kafka 1.0 and 2.0 using nifi-kafka-shared

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 3caad988be9d2e70ae5c1ee94d5c4a6bbe3c6b96
Author: exceptionfactory <ex...@apache.org>
AuthorDate: Tue Nov 22 16:08:22 2022 -0600

    NIFI-10866 Refactored Kafka 1.0 and 2.0 using nifi-kafka-shared
    
    Signed-off-by: Joe Gresock <jg...@gmail.com>
    
    This closes #6710.
---
 .../nifi-kafka-1-0-processors/pom.xml              |  13 +
 .../kafka/pubsub/ConsumeKafkaRecord_1_0.java       |  50 +-
 .../processors/kafka/pubsub/ConsumeKafka_1_0.java  |  58 ++-
 .../processors/kafka/pubsub/ConsumerLease.java     |  58 +--
 .../kafka/pubsub/KafkaProcessorUtils.java          | 405 ---------------
 .../kafka/pubsub/PublishKafkaRecord_1_0.java       |  72 ++-
 .../processors/kafka/pubsub/PublishKafka_1_0.java  |  63 +--
 .../record/sink/kafka/KafkaRecordSink_1_0.java     | 131 +----
 .../processors/kafka/pubsub/ConsumeKafkaTest.java  |  43 +-
 .../processors/kafka/pubsub/ConsumerPoolTest.java  |  15 +-
 .../processors/kafka/pubsub/ITConsumeKafka.java    |  12 +-
 .../kafka/pubsub/TestConsumeKafkaRecord_1_0.java   |  33 +-
 .../record/sink/kafka/TestKafkaRecordSink_1_0.java |  12 +-
 .../src/test/resources/log4j.properties            |  21 -
 .../src/test/resources/zookeeper.properties        |  20 -
 .../nifi-kafka-2-0-processors/pom.xml              |  14 +-
 .../kafka/pubsub/ConsumeKafkaRecord_2_0.java       |  61 +--
 .../processors/kafka/pubsub/ConsumeKafka_2_0.java  |  65 ++-
 .../processors/kafka/pubsub/ConsumerLease.java     |  41 +-
 .../kafka/pubsub/KafkaProcessorUtils.java          | 557 ---------------------
 .../kafka/pubsub/PublishKafkaRecord_2_0.java       |  83 ++-
 .../processors/kafka/pubsub/PublishKafka_2_0.java  |  73 +--
 .../record/sink/kafka/KafkaRecordSink_2_0.java     | 128 +----
 .../kafka/pubsub/ITConsumeKafka_2_0.java           |  12 +-
 .../kafka/pubsub/TestConsumeKafkaRecord_2_0.java   |  64 ++-
 .../kafka/pubsub/TestConsumeKafka_2_0.java         |  43 +-
 .../kafka/pubsub/TestConsumerPartitionsUtil.java   |   2 +-
 .../kafka/pubsub/TestPublishKafkaRecord_2_0.java   |   9 +-
 .../kafka/pubsub/TestPublishKafka_2_0.java         |   7 +-
 .../record/sink/kafka/TestKafkaRecordSink_2_0.java |  14 +-
 .../src/test/resources/log4j.properties            |  21 -
 31 files changed, 515 insertions(+), 1685 deletions(-)

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 cc3ec5d076..4711fcf04a 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
@@ -56,6 +56,19 @@
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-kerberos-credentials-service-api</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-kerberos-user-service-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-kerberos</artifactId>
+            <version>1.19.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-kafka-shared</artifactId>
+        </dependency>
         <dependency>
             <groupId>org.apache.kafka</groupId>
             <artifactId>kafka-clients</artifactId>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_1_0.java
index 28a582c08b..1dbe5ed3dd 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_1_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_1_0.java
@@ -20,7 +20,6 @@ import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -46,6 +45,12 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute;
+import org.apache.nifi.kafka.shared.component.KafkaClientComponent;
+import org.apache.nifi.kafka.shared.property.provider.KafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.property.provider.StandardKafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.validation.DynamicPropertyValidator;
+import org.apache.nifi.kafka.shared.validation.KafkaClientCustomValidationFunction;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -67,9 +72,9 @@ import org.apache.nifi.serialization.RecordSetWriterFactory;
 @WritesAttributes({
     @WritesAttribute(attribute = "record.count", description = "The number of records received"),
     @WritesAttribute(attribute = "mime.type", description = "The MIME Type that is provided by the configured Record Writer"),
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_PARTITION, description = "The partition of the topic the records are from"),
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_TIMESTAMP, description = "The timestamp of the message in the partition of the topic."),
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_TOPIC, description = "The topic records are from")
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_PARTITION, description = "The partition of the topic the records are from"),
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_TIMESTAMP, description = "The timestamp of the message in the partition of the topic."),
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_TOPIC, description = "The topic records are from")
 })
 @InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
 @DynamicProperty(name = "The name of a Kafka configuration property.", value = "The value of a given Kafka configuration property.",
@@ -78,7 +83,7 @@ import org.apache.nifi.serialization.RecordSetWriterFactory;
         + " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration. ",
         expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY)
 @SeeAlso({ConsumeKafka_1_0.class, PublishKafka_1_0.class, PublishKafkaRecord_1_0.class})
-public class ConsumeKafkaRecord_1_0 extends AbstractProcessor {
+public class ConsumeKafkaRecord_1_0 extends AbstractProcessor implements KafkaClientComponent {
 
     static final AllowableValue OFFSET_EARLIEST = new AllowableValue("earliest", "earliest", "Automatically reset the offset to the earliest offset");
     static final AllowableValue OFFSET_LATEST = new AllowableValue("latest", "latest", "Automatically reset the offset to the latest offset");
@@ -216,18 +221,19 @@ public class ConsumeKafkaRecord_1_0 extends AbstractProcessor {
 
     static {
         List<PropertyDescriptor> descriptors = new ArrayList<>();
-        descriptors.add(KafkaProcessorUtils.BOOTSTRAP_SERVERS);
+        descriptors.add(BOOTSTRAP_SERVERS);
         descriptors.add(TOPICS);
         descriptors.add(TOPIC_TYPE);
         descriptors.add(RECORD_READER);
         descriptors.add(RECORD_WRITER);
         descriptors.add(HONOR_TRANSACTIONS);
-        descriptors.add(KafkaProcessorUtils.SECURITY_PROTOCOL);
-        descriptors.add(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE);
-        descriptors.add(KafkaProcessorUtils.JAAS_SERVICE_NAME);
-        descriptors.add(KafkaProcessorUtils.USER_PRINCIPAL);
-        descriptors.add(KafkaProcessorUtils.USER_KEYTAB);
-        descriptors.add(KafkaProcessorUtils.SSL_CONTEXT_SERVICE);
+        descriptors.add(SECURITY_PROTOCOL);
+        descriptors.add(SASL_MECHANISM);
+        descriptors.add(KERBEROS_CREDENTIALS_SERVICE);
+        descriptors.add(KERBEROS_SERVICE_NAME);
+        descriptors.add(KERBEROS_PRINCIPAL);
+        descriptors.add(KERBEROS_KEYTAB);
+        descriptors.add(SSL_CONTEXT_SERVICE);
         descriptors.add(GROUP_ID);
         descriptors.add(AUTO_OFFSET_RESET);
         descriptors.add(MESSAGE_HEADER_ENCODING);
@@ -267,7 +273,7 @@ public class ConsumeKafkaRecord_1_0 extends AbstractProcessor {
         return new PropertyDescriptor.Builder()
                 .description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.")
                 .name(propertyDescriptorName)
-                .addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ConsumerConfig.class))
+                .addValidator(new DynamicPropertyValidator(ConsumerConfig.class))
                 .dynamic(true)
                 .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                 .build();
@@ -275,7 +281,7 @@ public class ConsumeKafkaRecord_1_0 extends AbstractProcessor {
 
     @Override
     protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
-        return KafkaProcessorUtils.validateCommonProperties(validationContext);
+        return new KafkaClientCustomValidationFunction().apply(validationContext);
     }
 
     private synchronized ConsumerPool getConsumerPool(final ProcessContext context) {
@@ -291,16 +297,16 @@ public class ConsumeKafkaRecord_1_0 extends AbstractProcessor {
         final int maxLeases = context.getMaxConcurrentTasks();
         final long maxUncommittedTime = context.getProperty(MAX_UNCOMMITTED_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
 
-        final Map<String, Object> props = new HashMap<>();
-        KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props);
+        final KafkaPropertyProvider kafkaPropertyProvider = new StandardKafkaPropertyProvider(ConsumerConfig.class);
+        final Map<String, Object> props = kafkaPropertyProvider.getProperties(context);
         props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
         props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
         props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
         final String topicListing = context.getProperty(ConsumeKafkaRecord_1_0.TOPICS).evaluateAttributeExpressions().getValue();
         final String topicType = context.getProperty(ConsumeKafkaRecord_1_0.TOPIC_TYPE).evaluateAttributeExpressions().getValue();
         final List<String> topics = new ArrayList<>();
-        final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue();
-        final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
+        final String securityProtocol = context.getProperty(SECURITY_PROTOCOL).getValue();
+        final String bootstrapServers = context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
 
         final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
         final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
@@ -327,7 +333,7 @@ public class ConsumeKafkaRecord_1_0 extends AbstractProcessor {
             return new ConsumerPool(maxLeases, readerFactory, writerFactory, props, topicPattern, maxUncommittedTime, securityProtocol,
                 bootstrapServers, log, honorTransactions, charset, headerNamePattern);
         } else {
-            getLogger().error("Subscription type has an unknown value {}", new Object[] {topicType});
+            getLogger().error("Subscription type has an unknown value {}", topicType);
             return null;
         }
     }
@@ -352,12 +358,12 @@ public class ConsumeKafkaRecord_1_0 extends AbstractProcessor {
         if (!activeLeases.isEmpty()) {
             int count = 0;
             for (final ConsumerLease lease : activeLeases) {
-                getLogger().info("Consumer {} has not finished after waiting 30 seconds; will attempt to wake-up the lease", new Object[] {lease});
+                getLogger().info("Consumer {} has not finished after waiting 30 seconds; will attempt to wake-up the lease", lease);
                 lease.wakeup();
                 count++;
             }
 
-            getLogger().info("Woke up {} consumers", new Object[] {count});
+            getLogger().info("Woke up {} consumers", count);
         }
 
         activeLeases.clear();
@@ -387,7 +393,7 @@ public class ConsumeKafkaRecord_1_0 extends AbstractProcessor {
                 }
             } catch (final WakeupException we) {
                 getLogger().warn("Was interrupted while trying to communicate with Kafka with lease {}. "
-                    + "Will roll back session and discard any partially received data.", new Object[] {lease});
+                    + "Will roll back session and discard any partially received data.", lease);
             } catch (final KafkaException kex) {
                 getLogger().error("Exception while interacting with Kafka so will close the lease {} due to {}",
                         new Object[]{lease, kex}, kex);
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_1_0.java
index 511d85fe7f..772f6cfa5e 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_1_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_1_0.java
@@ -21,7 +21,6 @@ import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -46,6 +45,13 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute;
+import org.apache.nifi.kafka.shared.component.KafkaClientComponent;
+import org.apache.nifi.kafka.shared.property.KeyEncoding;
+import org.apache.nifi.kafka.shared.property.provider.KafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.property.provider.StandardKafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.validation.DynamicPropertyValidator;
+import org.apache.nifi.kafka.shared.validation.KafkaClientCustomValidationFunction;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -53,20 +59,18 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING;
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING;
 
 @CapabilityDescription("Consumes messages from Apache Kafka specifically built against the Kafka 1.0 Consumer API. "
     + "The complementary NiFi processor for sending messages is PublishKafka_1_0.")
 @Tags({"Kafka", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume", "1.0"})
 @WritesAttributes({
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_COUNT, description = "The number of messages written if more than one"),
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_KEY, description = "The key of message if present and if single message. "
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_COUNT, description = "The number of messages written if more than one"),
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_KEY, description = "The key of message if present and if single message. "
             + "How the key is encoded depends on the value of the 'Key Attribute Encoding' property."),
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_OFFSET, description = "The offset of the message in the partition of the topic."),
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_TIMESTAMP, description = "The timestamp of the message in the partition of the topic."),
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_PARTITION, description = "The partition of the topic the message or message bundle is from"),
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_TOPIC, description = "The topic the message or message bundle is from")
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_OFFSET, description = "The offset of the message in the partition of the topic."),
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_TIMESTAMP, description = "The timestamp of the message in the partition of the topic."),
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_PARTITION, description = "The partition of the topic the message or message bundle is from"),
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_TOPIC, description = "The topic the message or message bundle is from")
 })
 @InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
 @DynamicProperty(name = "The name of a Kafka configuration property.", value = "The value of a given Kafka configuration property.",
@@ -74,7 +78,7 @@ import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_E
         + " In the event a dynamic property represents a property that was already set, its value will be ignored and WARN message logged."
         + " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration. ",
         expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY)
-public class ConsumeKafka_1_0 extends AbstractProcessor {
+public class ConsumeKafka_1_0 extends AbstractProcessor implements KafkaClientComponent {
 
     static final AllowableValue OFFSET_EARLIEST = new AllowableValue("earliest", "earliest", "Automatically reset the offset to the earliest offset");
 
@@ -126,10 +130,10 @@ public class ConsumeKafka_1_0 extends AbstractProcessor {
     static final PropertyDescriptor KEY_ATTRIBUTE_ENCODING = new PropertyDescriptor.Builder()
             .name("key-attribute-encoding")
             .displayName("Key Attribute Encoding")
-            .description("FlowFiles that are emitted have an attribute named '" + KafkaProcessorUtils.KAFKA_KEY + "'. This property dictates how the value of the attribute should be encoded.")
+            .description("FlowFiles that are emitted have an attribute named '" + KafkaFlowFileAttribute.KAFKA_KEY + "'. This property dictates how the value of the attribute should be encoded.")
             .required(true)
-            .defaultValue(UTF8_ENCODING.getValue())
-            .allowableValues(UTF8_ENCODING, HEX_ENCODING)
+            .defaultValue(KeyEncoding.UTF8.getValue())
+            .allowableValues(KeyEncoding.class)
             .build();
 
     static final PropertyDescriptor MESSAGE_DEMARCATOR = new PropertyDescriptor.Builder()
@@ -217,7 +221,13 @@ public class ConsumeKafka_1_0 extends AbstractProcessor {
 
     static {
         List<PropertyDescriptor> descriptors = new ArrayList<>();
-        descriptors.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors());
+        descriptors.add(BOOTSTRAP_SERVERS);
+        descriptors.add(SECURITY_PROTOCOL);
+        descriptors.add(SASL_MECHANISM);
+        descriptors.add(KERBEROS_SERVICE_NAME);
+        descriptors.add(KERBEROS_PRINCIPAL);
+        descriptors.add(KERBEROS_KEYTAB);
+        descriptors.add(SSL_CONTEXT_SERVICE);
         descriptors.add(TOPICS);
         descriptors.add(TOPIC_TYPE);
         descriptors.add(HONOR_TRANSACTIONS);
@@ -257,7 +267,7 @@ public class ConsumeKafka_1_0 extends AbstractProcessor {
         return new PropertyDescriptor.Builder()
                 .description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.")
                 .name(propertyDescriptorName)
-                .addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ConsumerConfig.class))
+                .addValidator(new DynamicPropertyValidator(ConsumerConfig.class))
                 .dynamic(true)
                 .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                 .build();
@@ -265,7 +275,7 @@ public class ConsumeKafka_1_0 extends AbstractProcessor {
 
     @Override
     protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
-        return KafkaProcessorUtils.validateCommonProperties(validationContext);
+        return new KafkaClientCustomValidationFunction().apply(validationContext);
     }
 
     private synchronized ConsumerPool getConsumerPool(final ProcessContext context) {
@@ -283,8 +293,8 @@ public class ConsumeKafka_1_0 extends AbstractProcessor {
         final byte[] demarcator = context.getProperty(ConsumeKafka_1_0.MESSAGE_DEMARCATOR).isSet()
                 ? context.getProperty(ConsumeKafka_1_0.MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8)
                 : null;
-        final Map<String, Object> props = new HashMap<>();
-        KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props);
+        final KafkaPropertyProvider kafkaPropertyProvider = new StandardKafkaPropertyProvider(ConsumerConfig.class);
+        final Map<String, Object> props = kafkaPropertyProvider.getProperties(context);
         props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
         props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
         props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
@@ -293,8 +303,8 @@ public class ConsumeKafka_1_0 extends AbstractProcessor {
         final String topicType = context.getProperty(ConsumeKafka_1_0.TOPIC_TYPE).evaluateAttributeExpressions().getValue();
         final List<String> topics = new ArrayList<>();
         final String keyEncoding = context.getProperty(KEY_ATTRIBUTE_ENCODING).getValue();
-        final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue();
-        final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
+        final String securityProtocol = context.getProperty(SECURITY_PROTOCOL).getValue();
+        final String bootstrapServers = context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
         final boolean honorTransactions = context.getProperty(HONOR_TRANSACTIONS).asBoolean();
 
         final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
@@ -318,7 +328,7 @@ public class ConsumeKafka_1_0 extends AbstractProcessor {
             return new ConsumerPool(maxLeases, demarcator, props, topicPattern, maxUncommittedTime, keyEncoding, securityProtocol,
                 bootstrapServers, log, honorTransactions, charset, headerNamePattern);
         } else {
-            getLogger().error("Subscription type has an unknown value {}", new Object[] {topicType});
+            getLogger().error("Subscription type has an unknown value {}", topicType);
             return null;
         }
     }
@@ -343,12 +353,12 @@ public class ConsumeKafka_1_0 extends AbstractProcessor {
         if (!activeLeases.isEmpty()) {
             int count = 0;
             for (final ConsumerLease lease : activeLeases) {
-                getLogger().info("Consumer {} has not finished after waiting 30 seconds; will attempt to wake-up the lease", new Object[] {lease});
+                getLogger().info("Consumer {} has not finished after waiting 30 seconds; will attempt to wake-up the lease", lease);
                 lease.wakeup();
                 count++;
             }
 
-            getLogger().info("Woke up {} consumers", new Object[] {count});
+            getLogger().info("Woke up {} consumers", count);
         }
 
         activeLeases.clear();
@@ -378,7 +388,7 @@ public class ConsumeKafka_1_0 extends AbstractProcessor {
                 }
             } catch (final WakeupException we) {
                 getLogger().warn("Was interrupted while trying to communicate with Kafka with lease {}. "
-                    + "Will roll back session and discard any partially received data.", new Object[] {lease});
+                    + "Will roll back session and discard any partially received data.", lease);
             } catch (final KafkaException kex) {
                 getLogger().error("Exception while interacting with Kafka so will close the lease {} due to {}",
                         new Object[]{lease, kex}, kex);
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
index 12aca509f0..6279b6ea49 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
@@ -27,6 +27,9 @@ import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.header.Header;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute;
+import org.apache.nifi.kafka.shared.attribute.StandardTransitUriProvider;
+import org.apache.nifi.kafka.shared.property.KeyEncoding;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.exception.ProcessException;
@@ -60,8 +63,6 @@ import java.util.stream.Collectors;
 
 import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_1_0.REL_PARSE_FAILURE;
 import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_1_0.REL_SUCCESS;
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING;
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING;
 
 /**
  * This class represents a lease to access a Kafka Consumer object. The lease is
@@ -142,7 +143,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
      */
     @Override
     public void onPartitionsRevoked(final Collection<TopicPartition> partitions) {
-        logger.debug("Rebalance Alert: Partitions '{}' revoked for lease '{}' with consumer '{}'", new Object[]{partitions, this, kafkaConsumer});
+        logger.debug("Rebalance Alert: Partitions '{}' revoked for lease '{}' with consumer '{}'", partitions, this, kafkaConsumer);
         //force a commit here.  Can reuse the session and consumer after this but must commit now to avoid duplicates if kafka reassigns partition
         commit();
     }
@@ -156,7 +157,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
      */
     @Override
     public void onPartitionsAssigned(final Collection<TopicPartition> partitions) {
-        logger.debug("Rebalance Alert: Partitions '{}' assigned for lease '{}' with consumer '{}'", new Object[]{partitions, this, kafkaConsumer});
+        logger.debug("Rebalance Alert: Partitions '{}' assigned for lease '{}' with consumer '{}'", partitions, this, kafkaConsumer);
     }
 
     /**
@@ -191,7 +192,6 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
      * higher performance than the other commitOffsets call as it allows the
      * kafka client to collect more data from Kafka before committing the
      * offsets.
-     *
      * if false then we didn't do anything and should probably yield if true
      * then we committed new data
      *
@@ -317,12 +317,12 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
     public abstract void yield();
 
     private void processRecords(final ConsumerRecords<byte[], byte[]> records) {
-        records.partitions().stream().forEach(partition -> {
+        records.partitions().forEach(partition -> {
             List<ConsumerRecord<byte[], byte[]>> messages = records.records(partition);
             if (!messages.isEmpty()) {
                 //update maximum offset map for this topic partition
                 long maxOffset = messages.stream()
-                        .mapToLong(record -> record.offset())
+                        .mapToLong(ConsumerRecord::offset)
                         .max()
                         .getAsLong();
 
@@ -332,9 +332,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
                 } else if (readerFactory != null && writerFactory != null) {
                     writeRecordData(getProcessSession(), messages, partition);
                 } else {
-                    messages.stream().forEach(message -> {
-                        writeData(getProcessSession(), message, partition);
-                    });
+                    messages.forEach(message -> writeData(getProcessSession(), message, partition));
                 }
 
                 totalMessages += messages.size();
@@ -348,9 +346,9 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
             return null;
         }
 
-        if (HEX_ENCODING.getValue().equals(encoding)) {
+        if (KeyEncoding.HEX.getValue().equals(encoding)) {
             return DatatypeConverter.printHexBinary(key);
-        } else if (UTF8_ENCODING.getValue().equals(encoding)) {
+        } else if (KeyEncoding.UTF8.getValue().equals(encoding)) {
             return new String(key, StandardCharsets.UTF_8);
         } else {
             return null;  // won't happen because it is guaranteed by the Allowable Values
@@ -384,8 +382,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
                 return false;
             }
 
-            final Map<String, String> attributes = new HashMap<>();
-            attributes.putAll(writeResult.getAttributes());
+            final Map<String, String> attributes = new HashMap<>(writeResult.getAttributes());
             attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType());
 
             bundle.flowFile = getProcessSession().putAllAttributes(bundle.flowFile, attributes);
@@ -401,9 +398,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
         tracker.incrementRecordCount(1);
         final byte[] value = record.value();
         if (value != null) {
-            flowFile = session.write(flowFile, out -> {
-                out.write(value);
-            });
+            flowFile = session.write(flowFile, out -> out.write(value));
         }
         flowFile = session.putAllAttributes(flowFile, getAttributes(record));
         tracker.updateFlowFile(flowFile);
@@ -464,10 +459,10 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
     private void handleParseFailure(final ConsumerRecord<byte[], byte[]> consumerRecord, final ProcessSession session, final Exception cause, final String message) {
         // If we are unable to parse the data, we need to transfer it to 'parse failure' relationship
         final Map<String, String> attributes = getAttributes(consumerRecord);
-        attributes.put(KafkaProcessorUtils.KAFKA_OFFSET, String.valueOf(consumerRecord.offset()));
-        attributes.put(KafkaProcessorUtils.KAFKA_TIMESTAMP, String.valueOf(consumerRecord.timestamp()));
-        attributes.put(KafkaProcessorUtils.KAFKA_PARTITION, String.valueOf(consumerRecord.partition()));
-        attributes.put(KafkaProcessorUtils.KAFKA_TOPIC, consumerRecord.topic());
+        attributes.put(KafkaFlowFileAttribute.KAFKA_OFFSET, String.valueOf(consumerRecord.offset()));
+        attributes.put(KafkaFlowFileAttribute.KAFKA_TIMESTAMP, String.valueOf(consumerRecord.timestamp()));
+        attributes.put(KafkaFlowFileAttribute.KAFKA_PARTITION, String.valueOf(consumerRecord.partition()));
+        attributes.put(KafkaFlowFileAttribute.KAFKA_TOPIC, consumerRecord.topic());
 
         FlowFile failureFlowFile = session.create();
 
@@ -477,7 +472,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
         }
         failureFlowFile = session.putAllAttributes(failureFlowFile, attributes);
 
-        final String transitUri = KafkaProcessorUtils.buildTransitURI(securityProtocol, bootstrapServers, consumerRecord.topic());
+        final String transitUri = StandardTransitUriProvider.getTransitUri(securityProtocol, bootstrapServers, consumerRecord.topic());
         session.getProvenanceReporter().receive(failureFlowFile, transitUri);
 
         session.transfer(failureFlowFile, REL_PARSE_FAILURE);
@@ -525,7 +520,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
                     try {
                         reader = readerFactory.createRecordReader(attributes, in, recordBytes.length, logger);
                     } catch (final IOException e) {
-                        yield();
+                        this.yield();
                         rollback(topicPartition);
                         handleParseFailure(consumerRecord, session, e, "Failed to parse message from Kafka due to comms failure. Will roll back session and try again momentarily.");
                         closeWriter(writer);
@@ -556,7 +551,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
                                     logger.error("Failed to obtain Schema for FlowFile. Will roll back the Kafka message offsets.", e);
 
                                     rollback(topicPartition);
-                                    yield();
+                                    this.yield();
 
                                     throw new ProcessException(e);
                                 }
@@ -584,7 +579,6 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
                         }
                     } catch (final IOException | MalformedRecordException | SchemaValidationException e) {
                         handleParseFailure(consumerRecord, session, e);
-                        continue;
                     }
                 }
             }
@@ -626,25 +620,25 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
 
     private void populateAttributes(final BundleTracker tracker) {
         final Map<String, String> kafkaAttrs = new HashMap<>();
-        kafkaAttrs.put(KafkaProcessorUtils.KAFKA_OFFSET, String.valueOf(tracker.initialOffset));
-        kafkaAttrs.put(KafkaProcessorUtils.KAFKA_TIMESTAMP, String.valueOf(tracker.initialTimestamp));
+        kafkaAttrs.put(KafkaFlowFileAttribute.KAFKA_OFFSET, String.valueOf(tracker.initialOffset));
+        kafkaAttrs.put(KafkaFlowFileAttribute.KAFKA_TIMESTAMP, String.valueOf(tracker.initialTimestamp));
         if (tracker.key != null && tracker.totalRecords == 1) {
-            kafkaAttrs.put(KafkaProcessorUtils.KAFKA_KEY, tracker.key);
+            kafkaAttrs.put(KafkaFlowFileAttribute.KAFKA_KEY, tracker.key);
         }
-        kafkaAttrs.put(KafkaProcessorUtils.KAFKA_PARTITION, String.valueOf(tracker.partition));
-        kafkaAttrs.put(KafkaProcessorUtils.KAFKA_TOPIC, tracker.topic);
+        kafkaAttrs.put(KafkaFlowFileAttribute.KAFKA_PARTITION, String.valueOf(tracker.partition));
+        kafkaAttrs.put(KafkaFlowFileAttribute.KAFKA_TOPIC, tracker.topic);
         if (tracker.totalRecords > 1) {
             // Add a record.count attribute to remain consistent with other record-oriented processors. If not
             // reading/writing records, then use "kafka.count" attribute.
             if (tracker.recordWriter == null) {
-                kafkaAttrs.put(KafkaProcessorUtils.KAFKA_COUNT, String.valueOf(tracker.totalRecords));
+                kafkaAttrs.put(KafkaFlowFileAttribute.KAFKA_COUNT, String.valueOf(tracker.totalRecords));
             } else {
                 kafkaAttrs.put("record.count", String.valueOf(tracker.totalRecords));
             }
         }
         final FlowFile newFlowFile = getProcessSession().putAllAttributes(tracker.flowFile, kafkaAttrs);
         final long executionDurationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - leaseStartNanos);
-        final String transitUri = KafkaProcessorUtils.buildTransitURI(securityProtocol, bootstrapServers, tracker.topic);
+        final String transitUri = StandardTransitUriProvider.getTransitUri(securityProtocol, bootstrapServers, tracker.topic);
         getProcessSession().getProvenanceReporter().receive(newFlowFile, transitUri, executionDurationMillis);
         tracker.updateFlowFile(newFlowFile);
     }
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
deleted file mode 100644
index 322d77efea..0000000000
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
+++ /dev/null
@@ -1,405 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.kafka.pubsub;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.kafka.clients.CommonClientConfigs;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.common.config.SaslConfigs;
-import org.apache.kafka.common.config.SslConfigs;
-import org.apache.kafka.common.serialization.ByteArrayDeserializer;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.apache.nifi.components.AllowableValue;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.components.Validator;
-import org.apache.nifi.components.resource.ResourceCardinality;
-import org.apache.nifi.components.resource.ResourceType;
-import org.apache.nifi.expression.ExpressionLanguageScope;
-import org.apache.nifi.kerberos.KerberosCredentialsService;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.ssl.SSLContextService;
-import org.apache.nifi.util.FormatUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.reflect.Field;
-import java.lang.reflect.Modifier;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Supplier;
-import java.util.regex.Pattern;
-
-public final class KafkaProcessorUtils {
-    private static final String ALLOW_EXPLICIT_KEYTAB = "NIFI_ALLOW_EXPLICIT_KEYTAB";
-
-    final Logger logger = LoggerFactory.getLogger(this.getClass());
-
-    static final AllowableValue UTF8_ENCODING = new AllowableValue("utf-8", "UTF-8 Encoded", "The key is interpreted as a UTF-8 Encoded string.");
-    static final AllowableValue HEX_ENCODING = new AllowableValue("hex", "Hex Encoded",
-            "The key is interpreted as arbitrary binary data and is encoded using hexadecimal characters with uppercase letters");
-
-    static final Pattern HEX_KEY_PATTERN = Pattern.compile("(?:[0123456789abcdefABCDEF]{2})+");
-
-    static final String KAFKA_KEY = "kafka.key";
-    static final String KAFKA_TOPIC = "kafka.topic";
-    static final String KAFKA_PARTITION = "kafka.partition";
-    static final String KAFKA_OFFSET = "kafka.offset";
-    static final String KAFKA_TIMESTAMP = "kafka.timestamp";
-    static final String KAFKA_COUNT = "kafka.count";
-    public static final AllowableValue SEC_PLAINTEXT = new AllowableValue("PLAINTEXT", "PLAINTEXT", "PLAINTEXT");
-    public static final AllowableValue SEC_SSL = new AllowableValue("SSL", "SSL", "SSL");
-    public static final AllowableValue SEC_SASL_PLAINTEXT = new AllowableValue("SASL_PLAINTEXT", "SASL_PLAINTEXT", "SASL_PLAINTEXT");
-    public static final AllowableValue SEC_SASL_SSL = new AllowableValue("SASL_SSL", "SASL_SSL", "SASL_SSL");
-
-    public static final PropertyDescriptor BOOTSTRAP_SERVERS = new PropertyDescriptor.Builder()
-            .name(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)
-            .displayName("Kafka Brokers")
-            .description("A comma-separated list of known Kafka Brokers in the format <host>:<port>")
-            .required(true)
-            .addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-            .defaultValue("localhost:9092")
-            .build();
-    public static final PropertyDescriptor SECURITY_PROTOCOL = new PropertyDescriptor.Builder()
-            .name("security.protocol")
-            .displayName("Security Protocol")
-            .description("Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.")
-            .required(true)
-            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .allowableValues(SEC_PLAINTEXT, SEC_SSL, SEC_SASL_PLAINTEXT, SEC_SASL_SSL)
-            .defaultValue(SEC_PLAINTEXT.getValue())
-            .build();
-    public static final PropertyDescriptor JAAS_SERVICE_NAME = new PropertyDescriptor.Builder()
-            .name("sasl.kerberos.service.name")
-            .displayName("Kerberos Service Name")
-            .description("The service name that matches the primary name of the Kafka server configured in the broker JAAS file."
-                    + "This can be defined either in Kafka's JAAS config or in Kafka's config. "
-                    + "Corresponds to Kafka's 'security.protocol' property."
-                    + "It is ignored unless one of the SASL options of the <Security Protocol> are selected.")
-            .required(false)
-            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-            .build();
-    static final PropertyDescriptor USER_PRINCIPAL = new PropertyDescriptor.Builder()
-            .name("sasl.kerberos.principal")
-            .displayName("Kerberos Principal")
-            .description("The Kerberos principal that will be used to connect to brokers. If not set, it is expected to set a JAAS configuration file "
-                    + "in the JVM properties defined in the bootstrap.conf file. This principal will be set into 'sasl.jaas.config' Kafka's property.")
-            .required(false)
-            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-            .build();
-    static final PropertyDescriptor USER_KEYTAB = new PropertyDescriptor.Builder()
-            .name("sasl.kerberos.keytab")
-            .displayName("Kerberos Keytab")
-            .description("The Kerberos keytab that will be used to connect to brokers. If not set, it is expected to set a JAAS configuration file "
-                    + "in the JVM properties defined in the bootstrap.conf file. This principal will be set into 'sasl.jaas.config' Kafka's property.")
-            .required(false)
-            .identifiesExternalResource(ResourceCardinality.SINGLE, ResourceType.FILE)
-            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-            .build();
-    public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
-            .name("ssl.context.service")
-            .displayName("SSL Context Service")
-            .description("Specifies the SSL Context Service to use for communicating with Kafka.")
-            .required(false)
-            .identifiesControllerService(SSLContextService.class)
-            .build();
-    public static final PropertyDescriptor KERBEROS_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
-        .name("kerberos-credentials-service")
-        .displayName("Kerberos Credentials Service")
-        .description("Specifies the Kerberos Credentials Controller Service that should be used for authenticating with Kerberos")
-        .identifiesControllerService(KerberosCredentialsService.class)
-        .required(false)
-        .build();
-
-    static List<PropertyDescriptor> getCommonPropertyDescriptors() {
-        return Arrays.asList(
-                BOOTSTRAP_SERVERS,
-                SECURITY_PROTOCOL,
-                JAAS_SERVICE_NAME,
-                KERBEROS_CREDENTIALS_SERVICE,
-                USER_PRINCIPAL,
-                USER_KEYTAB,
-                SSL_CONTEXT_SERVICE
-        );
-    }
-
-    public static Collection<ValidationResult> validateCommonProperties(final ValidationContext validationContext) {
-        List<ValidationResult> results = new ArrayList<>();
-
-        String securityProtocol = validationContext.getProperty(SECURITY_PROTOCOL).getValue();
-
-        final String explicitPrincipal = validationContext.getProperty(USER_PRINCIPAL).evaluateAttributeExpressions().getValue();
-        final String explicitKeytab = validationContext.getProperty(USER_KEYTAB).evaluateAttributeExpressions().getValue();
-        final KerberosCredentialsService credentialsService = validationContext.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
-
-        final String resolvedPrincipal;
-        final String resolvedKeytab;
-        if (credentialsService == null) {
-            resolvedPrincipal = explicitPrincipal;
-            resolvedKeytab = explicitKeytab;
-        } else {
-            resolvedPrincipal = credentialsService.getPrincipal();
-            resolvedKeytab = credentialsService.getKeytab();
-        }
-
-        if (credentialsService != null && (explicitPrincipal != null || explicitKeytab != null)) {
-            results.add(new ValidationResult.Builder()
-                .subject("Kerberos Credentials")
-                .valid(false)
-                .explanation("Cannot specify both a Kerberos Credentials Service and a principal/keytab")
-                .build());
-        }
-
-        final String allowExplicitKeytabVariable = System.getenv(ALLOW_EXPLICIT_KEYTAB);
-        if ("false".equalsIgnoreCase(allowExplicitKeytabVariable) && (explicitPrincipal != null || explicitKeytab != null)) {
-            results.add(new ValidationResult.Builder()
-                .subject("Kerberos Credentials")
-                .valid(false)
-                .explanation("The '" + ALLOW_EXPLICIT_KEYTAB + "' system environment variable is configured to forbid explicitly configuring principal/keytab in processors. "
-                    + "The Kerberos Credentials Service should be used instead of setting the Kerberos Keytab or Kerberos Principal property.")
-                .build());
-        }
-
-        // validates that if one of SASL (Kerberos) option is selected for
-        // security protocol, then Kerberos principal is provided as well
-        if (SEC_SASL_PLAINTEXT.getValue().equals(securityProtocol) || SEC_SASL_SSL.getValue().equals(securityProtocol)) {
-            String jaasServiceName = validationContext.getProperty(JAAS_SERVICE_NAME).evaluateAttributeExpressions().getValue();
-            if (jaasServiceName == null || jaasServiceName.trim().length() == 0) {
-                results.add(new ValidationResult.Builder().subject(JAAS_SERVICE_NAME.getDisplayName()).valid(false)
-                    .explanation("The <" + JAAS_SERVICE_NAME.getDisplayName() + "> property must be set when <"
-                        + SECURITY_PROTOCOL.getDisplayName() + "> is configured as '"
-                        + SEC_SASL_PLAINTEXT.getValue() + "' or '" + SEC_SASL_SSL.getValue() + "'.")
-                    .build());
-            }
-
-            if ((resolvedKeytab == null && resolvedPrincipal != null) || (resolvedKeytab != null && resolvedPrincipal == null)) {
-                results.add(new ValidationResult.Builder()
-                    .subject(JAAS_SERVICE_NAME.getDisplayName())
-                    .valid(false)
-                    .explanation("Both <" + USER_KEYTAB.getDisplayName() + "> and <" + USER_PRINCIPAL.getDisplayName() + "> "
-                        + "must be set or neither must be set.")
-                    .build());
-            }
-        }
-
-        // If SSL or SASL_SSL then SSLContext Controller Service must be set.
-        final boolean sslProtocol = SEC_SSL.getValue().equals(securityProtocol) || SEC_SASL_SSL.getValue().equals(securityProtocol);
-        final boolean csSet = validationContext.getProperty(SSL_CONTEXT_SERVICE).isSet();
-        if (csSet && !sslProtocol) {
-            results.add(new ValidationResult.Builder()
-                .subject(SECURITY_PROTOCOL.getDisplayName())
-                .valid(false)
-                .explanation("If you set the SSL Controller Service you should also choose an SSL based security protocol.")
-                .build());
-        }
-
-        if (!csSet && sslProtocol) {
-            results.add(new ValidationResult.Builder()
-                .subject(SSL_CONTEXT_SERVICE.getDisplayName())
-                .valid(false)
-                .explanation("If you set to an SSL based protocol you need to set the SSL Controller Service")
-                .build());
-        }
-
-        final String enableAutoCommit = validationContext.getProperty(new PropertyDescriptor.Builder().name(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG).build()).getValue();
-        if (enableAutoCommit != null && !enableAutoCommit.toLowerCase().equals("false")) {
-            results.add(new ValidationResult.Builder().subject(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)
-                .explanation("Enable auto commit must be false. It is managed by the processor.").build());
-        }
-
-        final String keySerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG).build()).getValue();
-        if (keySerializer != null && !ByteArraySerializer.class.getName().equals(keySerializer)) {
-            results.add(new ValidationResult.Builder().subject(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)
-                .explanation("Key Serializer must be " + ByteArraySerializer.class.getName() + "' was '" + keySerializer + "'").build());
-        }
-
-        final String valueSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG).build()).getValue();
-        if (valueSerializer != null && !ByteArraySerializer.class.getName().equals(valueSerializer)) {
-            results.add(new ValidationResult.Builder().subject(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)
-                .explanation("Value Serializer must be " + ByteArraySerializer.class.getName() + "' was '" + valueSerializer + "'").build());
-        }
-
-        final String keyDeSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG).build()).getValue();
-        if (keyDeSerializer != null && !ByteArrayDeserializer.class.getName().equals(keyDeSerializer)) {
-            results.add(new ValidationResult.Builder().subject(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)
-                .explanation("Key De-Serializer must be '" + ByteArrayDeserializer.class.getName() + "' was '" + keyDeSerializer + "'").build());
-        }
-
-        final String valueDeSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG).build()).getValue();
-        if (valueDeSerializer != null && !ByteArrayDeserializer.class.getName().equals(valueDeSerializer)) {
-            results.add(new ValidationResult.Builder().subject(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG)
-                .explanation("Value De-Serializer must be " + ByteArrayDeserializer.class.getName() + "' was '" + valueDeSerializer + "'").build());
-        }
-
-        return results;
-    }
-
-    public static final class KafkaConfigValidator implements Validator {
-
-        final Class<?> classType;
-
-        public KafkaConfigValidator(final Class<?> classType) {
-            this.classType = classType;
-        }
-
-        @Override
-        public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
-            final boolean knownValue = KafkaProcessorUtils.isStaticStringFieldNamePresent(subject, classType, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class);
-            return new ValidationResult.Builder().subject(subject).explanation("Must be a known configuration parameter for this kafka client").valid(knownValue).build();
-        }
-    }
-
-    /**
-     * Builds transit URI for provenance event. The transit URI will be in the
-     * form of &lt;security.protocol&gt;://&lt;bootstrap.servers&gt;/topic
-     */
-    static String buildTransitURI(String securityProtocol, String brokers, String topic) {
-        StringBuilder builder = new StringBuilder();
-        builder.append(securityProtocol);
-        builder.append("://");
-        builder.append(brokers);
-        builder.append("/");
-        builder.append(topic);
-        return builder.toString();
-    }
-
-
-    static void buildCommonKafkaProperties(final ProcessContext context, final Class<?> kafkaConfigClass, final Map<String, Object> mapToPopulate) {
-        for (PropertyDescriptor propertyDescriptor : context.getProperties().keySet()) {
-            if (propertyDescriptor.equals(SSL_CONTEXT_SERVICE)) {
-                // Translate SSLContext Service configuration into Kafka properties
-                final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
-                if (sslContextService != null && sslContextService.isKeyStoreConfigured()) {
-                    mapToPopulate.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, sslContextService.getKeyStoreFile());
-                    mapToPopulate.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, sslContextService.getKeyStorePassword());
-                    final String keyPass = sslContextService.getKeyPassword() == null ? sslContextService.getKeyStorePassword() : sslContextService.getKeyPassword();
-                    mapToPopulate.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, keyPass);
-                    mapToPopulate.put(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, sslContextService.getKeyStoreType());
-                }
-
-                if (sslContextService != null && sslContextService.isTrustStoreConfigured()) {
-                    mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, sslContextService.getTrustStoreFile());
-                    mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, sslContextService.getTrustStorePassword());
-                    mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, sslContextService.getTrustStoreType());
-                }
-            }
-
-            String propertyName = propertyDescriptor.getName();
-            String propertyValue = propertyDescriptor.isExpressionLanguageSupported()
-                    ? context.getProperty(propertyDescriptor).evaluateAttributeExpressions().getValue()
-                    : context.getProperty(propertyDescriptor).getValue();
-
-            if (propertyValue != null && !propertyName.equals(USER_PRINCIPAL.getName()) && !propertyName.equals(USER_KEYTAB.getName())) {
-                // If the property name ends in ".ms" then it is a time period. We want to accept either an integer as number of milliseconds
-                // or the standard NiFi time period such as "5 secs"
-                if (propertyName.endsWith(".ms") && !StringUtils.isNumeric(propertyValue.trim())) { // kafka standard time notation
-                    propertyValue = String.valueOf(FormatUtils.getTimeDuration(propertyValue.trim(), TimeUnit.MILLISECONDS));
-                }
-
-                if (isStaticStringFieldNamePresent(propertyName, kafkaConfigClass, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class)) {
-                    mapToPopulate.put(propertyName, propertyValue);
-                }
-            }
-        }
-
-        String securityProtocol = context.getProperty(SECURITY_PROTOCOL).getValue();
-        if (SEC_SASL_PLAINTEXT.getValue().equals(securityProtocol) || SEC_SASL_SSL.getValue().equals(securityProtocol)) {
-            setJaasConfig(mapToPopulate, context);
-        }
-    }
-
-    /**
-     * Method used to create a transactional id Supplier for KafkaProducer
-     *
-     * @param prefix String transactional id prefix, can be null
-     * @return A Supplier that generates transactional id
-     */
-    public static Supplier<String> getTransactionalIdSupplier(String prefix) {
-        return () -> (prefix == null ? "" : prefix)  + UUID.randomUUID().toString();
-    }
-
-    /**
-     * Method used to configure the 'sasl.jaas.config' property based on KAFKA-4259<br />
-     * https://cwiki.apache.org/confluence/display/KAFKA/KIP-85%3A+Dynamic+JAAS+configuration+for+Kafka+clients<br />
-     * <br />
-     * It expects something with the following format: <br />
-     * <br />
-     * &lt;LoginModuleClass&gt; &lt;ControlFlag&gt; *(&lt;OptionName&gt;=&lt;OptionValue&gt;); <br />
-     * ControlFlag = required / requisite / sufficient / optional
-     *
-     * @param mapToPopulate Map of configuration properties
-     * @param context Context
-     */
-    private static void setJaasConfig(Map<String, Object> mapToPopulate, ProcessContext context) {
-        String keytab = context.getProperty(USER_KEYTAB) == null ? null : context.getProperty(USER_KEYTAB).evaluateAttributeExpressions().getValue();
-        String principal = context.getProperty(USER_PRINCIPAL) == null ? null : context.getProperty(USER_PRINCIPAL).evaluateAttributeExpressions().getValue();
-
-        // If the Kerberos Credentials Service is specified, we need to use its configuration, not the explicit properties for principal/keytab.
-        // The customValidate method ensures that only one can be set, so we know that the principal & keytab above are null.
-        final KerberosCredentialsService credentialsService = context.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
-        if (credentialsService != null) {
-            principal = credentialsService.getPrincipal();
-            keytab = credentialsService.getKeytab();
-        }
-
-
-        String serviceName = context.getProperty(JAAS_SERVICE_NAME).evaluateAttributeExpressions().getValue();
-        if(StringUtils.isNotBlank(keytab) && StringUtils.isNotBlank(principal) && StringUtils.isNotBlank(serviceName)) {
-            mapToPopulate.put(SaslConfigs.SASL_JAAS_CONFIG, "com.sun.security.auth.module.Krb5LoginModule required "
-                    + "useTicketCache=false "
-                    + "renewTicket=true "
-                    + "serviceName=\"" + serviceName + "\" "
-                    + "useKeyTab=true "
-                    + "keyTab=\"" + keytab + "\" "
-                    + "principal=\"" + principal + "\";");
-        }
-    }
-
-    public static boolean isStaticStringFieldNamePresent(final String name, final Class<?>... classes) {
-        return KafkaProcessorUtils.getPublicStaticStringFieldValues(classes).contains(name);
-    }
-
-    private static Set<String> getPublicStaticStringFieldValues(final Class<?>... classes) {
-        final Set<String> strings = new HashSet<>();
-        for (final Class<?> classType : classes) {
-            for (final Field field : classType.getDeclaredFields()) {
-                if (Modifier.isPublic(field.getModifiers()) && Modifier.isStatic(field.getModifiers()) && field.getType().equals(String.class)) {
-                    try {
-                        strings.add(String.valueOf(field.get(null)));
-                    } catch (IllegalArgumentException | IllegalAccessException ex) {
-                        //ignore
-                    }
-                }
-            }
-        }
-        return strings;
-    }
-
-}
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_1_0.java
index 1edd5bac67..69805eb706 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_1_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_1_0.java
@@ -32,13 +32,19 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.kafka.shared.attribute.StandardTransitUriProvider;
+import org.apache.nifi.kafka.shared.component.KafkaClientComponent;
+import org.apache.nifi.kafka.shared.property.provider.KafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.property.provider.StandardKafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.transaction.TransactionIdSupplier;
+import org.apache.nifi.kafka.shared.validation.DynamicPropertyValidator;
+import org.apache.nifi.kafka.shared.validation.KafkaClientCustomValidationFunction;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.util.FlowFileFilters;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.record.path.RecordPath;
@@ -54,13 +60,10 @@ import org.apache.nifi.serialization.record.Record;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.RecordSet;
 
-import java.io.IOException;
-import java.io.InputStream;
 import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -88,7 +91,7 @@ import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIB
 @WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Kafka for this FlowFile. This attribute is added only to "
     + "FlowFiles that are routed to success.")
 @SeeAlso({PublishKafka_1_0.class, ConsumeKafka_1_0.class, ConsumeKafkaRecord_1_0.class})
-public class PublishKafkaRecord_1_0 extends AbstractProcessor {
+public class PublishKafkaRecord_1_0 extends AbstractProcessor implements KafkaClientComponent {
     protected static final String MSG_COUNT = "msg.count";
 
     static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("all", "Guarantee Replicated Delivery",
@@ -115,10 +118,6 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
         "Interprets the <Partition> property as Expression Language that will be evaluated against each FlowFile. This Expression will be evaluated once against the FlowFile, " +
             "so all Records in a given FlowFile will go to the same partition.");
 
-    static final AllowableValue UTF8_ENCODING = new AllowableValue("utf-8", "UTF-8 Encoded", "The key is interpreted as a UTF-8 Encoded string.");
-    static final AllowableValue HEX_ENCODING = new AllowableValue("hex", "Hex Encoded",
-        "The key is interpreted as arbitrary binary data that is encoded using hexadecimal characters with uppercase letters.");
-
     static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder()
         .name("topic")
         .displayName("Topic Name")
@@ -282,7 +281,7 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
 
     static {
         final List<PropertyDescriptor> properties = new ArrayList<>();
-        properties.add(KafkaProcessorUtils.BOOTSTRAP_SERVERS);
+        properties.add(BOOTSTRAP_SERVERS);
         properties.add(TOPIC);
         properties.add(RECORD_READER);
         properties.add(RECORD_WRITER);
@@ -291,12 +290,12 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
         properties.add(DELIVERY_GUARANTEE);
         properties.add(ATTRIBUTE_NAME_REGEX);
         properties.add(MESSAGE_HEADER_ENCODING);
-        properties.add(KafkaProcessorUtils.SECURITY_PROTOCOL);
-        properties.add(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE);
-        properties.add(KafkaProcessorUtils.JAAS_SERVICE_NAME);
-        properties.add(KafkaProcessorUtils.USER_PRINCIPAL);
-        properties.add(KafkaProcessorUtils.USER_KEYTAB);
-        properties.add(KafkaProcessorUtils.SSL_CONTEXT_SERVICE);
+        properties.add(SECURITY_PROTOCOL);
+        properties.add(KERBEROS_CREDENTIALS_SERVICE);
+        properties.add(KERBEROS_SERVICE_NAME);
+        properties.add(KERBEROS_PRINCIPAL);
+        properties.add(KERBEROS_KEYTAB);
+        properties.add(SSL_CONTEXT_SERVICE);
         properties.add(MESSAGE_KEY_FIELD);
         properties.add(MAX_REQUEST_SIZE);
         properties.add(ACK_WAIT_TIME);
@@ -328,7 +327,7 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
         return new PropertyDescriptor.Builder()
             .description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.")
             .name(propertyDescriptorName)
-            .addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class))
+            .addValidator(new DynamicPropertyValidator(ProducerConfig.class))
             .dynamic(true)
             .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
@@ -336,8 +335,7 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
 
     @Override
     protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
-        final List<ValidationResult> results = new ArrayList<>();
-        results.addAll(KafkaProcessorUtils.validateCommonProperties(validationContext));
+        final List<ValidationResult> results = new ArrayList<>(new KafkaClientCustomValidationFunction().apply(validationContext));
 
         final boolean useTransactions = validationContext.getProperty(USE_TRANSACTIONS).asBoolean();
         if (useTransactions) {
@@ -392,19 +390,19 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
 
     protected PublisherPool createPublisherPool(final ProcessContext context) {
         final int maxMessageSize = context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue();
-        final long maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS).longValue();
+        final long maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
 
         final String attributeNameRegex = context.getProperty(ATTRIBUTE_NAME_REGEX).getValue();
         final Pattern attributeNamePattern = attributeNameRegex == null ? null : Pattern.compile(attributeNameRegex);
         final boolean useTransactions = context.getProperty(USE_TRANSACTIONS).asBoolean();
         final String transactionalIdPrefix = context.getProperty(TRANSACTIONAL_ID_PREFIX).evaluateAttributeExpressions().getValue();
-        Supplier<String> transactionalIdSupplier = KafkaProcessorUtils.getTransactionalIdSupplier(transactionalIdPrefix);
+        Supplier<String> transactionalIdSupplier = new TransactionIdSupplier(transactionalIdPrefix);
 
         final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
         final Charset charset = Charset.forName(charsetName);
 
-        final Map<String, Object> kafkaProperties = new HashMap<>();
-        KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
+        final KafkaPropertyProvider propertyProvider = new StandardKafkaPropertyProvider(ProducerConfig.class);
+        final Map<String, Object> kafkaProperties = propertyProvider.getProperties(context);
         kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize));
@@ -434,8 +432,8 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
             return;
         }
 
-        final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue();
-        final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
+        final String securityProtocol = context.getProperty(SECURITY_PROTOCOL).getValue();
+        final String bootstrapServers = context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
         final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
         final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
         final boolean useTransactions = context.getProperty(USE_TRANSACTIONS).asBoolean();
@@ -470,24 +468,20 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
                 final Function<Record, Integer> partitioner = getPartitioner(context, flowFile);
 
                 try {
-                    session.read(flowFile, new InputStreamCallback() {
-                        @Override
-                        public void process(final InputStream in) throws IOException {
-                            try {
-                                final RecordReader reader = readerFactory.createRecordReader(flowFile, in, getLogger());
-                                final RecordSet recordSet = reader.createRecordSet();
-
-                                final RecordSchema schema = writerFactory.getSchema(flowFile.getAttributes(), recordSet.getSchema());
-                                lease.publish(flowFile, recordSet, writerFactory, schema, messageKeyField, topic, partitioner);
-                            } catch (final SchemaNotFoundException | MalformedRecordException e) {
-                                throw new ProcessException(e);
-                            }
+                    session.read(flowFile, in -> {
+                        try {
+                            final RecordReader reader = readerFactory.createRecordReader(flowFile, in, getLogger());
+                            final RecordSet recordSet = reader.createRecordSet();
+
+                            final RecordSchema schema = writerFactory.getSchema(flowFile.getAttributes(), recordSet.getSchema());
+                            lease.publish(flowFile, recordSet, writerFactory, schema, messageKeyField, topic, partitioner);
+                        } catch (final SchemaNotFoundException | MalformedRecordException e) {
+                            throw new ProcessException(e);
                         }
                     });
                 } catch (final Exception e) {
                     // The FlowFile will be obtained and the error logged below, when calling publishResult.getFailedFlowFiles()
                     lease.fail(flowFile, e);
-                    continue;
                 }
             }
 
@@ -509,7 +503,7 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
                 success = session.putAttribute(success, MSG_COUNT, String.valueOf(msgCount));
                 session.adjustCounter("Messages Sent", msgCount, true);
 
-                final String transitUri = KafkaProcessorUtils.buildTransitURI(securityProtocol, bootstrapServers, topic);
+                final String transitUri = StandardTransitUriProvider.getTransitUri(securityProtocol, bootstrapServers, topic);
                 session.getProvenanceReporter().send(success, transitUri, "Sent " + msgCount + " messages", transmissionMillis);
                 session.transfer(success, REL_SUCCESS);
             }
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_1_0.java
index a0e0ecb169..6de516be7d 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_1_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_1_0.java
@@ -31,26 +31,32 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute;
+import org.apache.nifi.kafka.shared.attribute.StandardTransitUriProvider;
+import org.apache.nifi.kafka.shared.component.KafkaPublishComponent;
+import org.apache.nifi.kafka.shared.property.KeyEncoding;
+import org.apache.nifi.kafka.shared.property.provider.KafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.property.provider.StandardKafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.transaction.TransactionIdSupplier;
+import org.apache.nifi.kafka.shared.validation.DynamicPropertyValidator;
+import org.apache.nifi.kafka.shared.validation.KafkaClientCustomValidationFunction;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.util.FlowFileFilters;
 import org.apache.nifi.processor.util.StandardValidators;
 
 import javax.xml.bind.DatatypeConverter;
 import java.io.BufferedInputStream;
-import java.io.IOException;
 import java.io.InputStream;
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -76,7 +82,7 @@ import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIB
 @WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Kafka for this FlowFile. This attribute is added only to "
     + "FlowFiles that are routed to success. If the <Message Demarcator> Property is not set, this will always be 1, but if the Property is set, it may "
     + "be greater than 1.")
-public class PublishKafka_1_0 extends AbstractProcessor {
+public class PublishKafka_1_0 extends AbstractProcessor implements KafkaPublishComponent {
     protected static final String MSG_COUNT = "msg.count";
 
     static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("all", "Guarantee Replicated Delivery",
@@ -100,10 +106,6 @@ public class PublishKafka_1_0 extends AbstractProcessor {
         "Interprets the <Partition> property as Expression Language that will be evaluated against each FlowFile. This Expression will be evaluated once against the FlowFile, " +
             "so all Records in a given FlowFile will go to the same partition.");
 
-    static final AllowableValue UTF8_ENCODING = new AllowableValue("utf-8", "UTF-8 Encoded", "The key is interpreted as a UTF-8 Encoded string.");
-    static final AllowableValue HEX_ENCODING = new AllowableValue("hex", "Hex Encoded",
-        "The key is interpreted as arbitrary binary data that is encoded using hexadecimal characters with uppercase letters.");
-
     static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder()
         .name("topic")
         .displayName("Topic Name")
@@ -170,10 +172,10 @@ public class PublishKafka_1_0 extends AbstractProcessor {
     static final PropertyDescriptor KEY_ATTRIBUTE_ENCODING = new PropertyDescriptor.Builder()
         .name("key-attribute-encoding")
         .displayName("Key Attribute Encoding")
-        .description("FlowFiles that are emitted have an attribute named '" + KafkaProcessorUtils.KAFKA_KEY + "'. This property dictates how the value of the attribute should be encoded.")
+        .description("FlowFiles that are emitted have an attribute named '" + KafkaFlowFileAttribute.KAFKA_KEY + "'. This property dictates how the value of the attribute should be encoded.")
         .required(true)
-        .defaultValue(UTF8_ENCODING.getValue())
-        .allowableValues(UTF8_ENCODING, HEX_ENCODING)
+        .defaultValue(KeyEncoding.UTF8.getValue())
+        .allowableValues(KeyEncoding.class)
         .build();
 
     static final PropertyDescriptor MESSAGE_DEMARCATOR = new PropertyDescriptor.Builder()
@@ -273,7 +275,12 @@ public class PublishKafka_1_0 extends AbstractProcessor {
 
     static {
         final List<PropertyDescriptor> properties = new ArrayList<>();
-        properties.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors());
+        properties.add(BOOTSTRAP_SERVERS);
+        properties.add(SECURITY_PROTOCOL);
+        properties.add(KERBEROS_SERVICE_NAME);
+        properties.add(KERBEROS_PRINCIPAL);
+        properties.add(KERBEROS_KEYTAB);
+        properties.add(SSL_CONTEXT_SERVICE);
         properties.add(TOPIC);
         properties.add(DELIVERY_GUARANTEE);
         properties.add(USE_TRANSACTIONS);
@@ -313,7 +320,7 @@ public class PublishKafka_1_0 extends AbstractProcessor {
         return new PropertyDescriptor.Builder()
             .description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.")
             .name(propertyDescriptorName)
-            .addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class))
+            .addValidator(new DynamicPropertyValidator(ProducerConfig.class))
             .dynamic(true)
             .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
@@ -321,8 +328,7 @@ public class PublishKafka_1_0 extends AbstractProcessor {
 
     @Override
     protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
-        final List<ValidationResult> results = new ArrayList<>();
-        results.addAll(KafkaProcessorUtils.validateCommonProperties(validationContext));
+        final List<ValidationResult> results = new ArrayList<>(new KafkaClientCustomValidationFunction().apply(validationContext));
 
         final boolean useTransactions = validationContext.getProperty(USE_TRANSACTIONS).asBoolean();
         if (useTransactions) {
@@ -363,19 +369,19 @@ public class PublishKafka_1_0 extends AbstractProcessor {
 
     protected PublisherPool createPublisherPool(final ProcessContext context) {
         final int maxMessageSize = context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue();
-        final long maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS).longValue();
+        final long maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
 
         final String attributeNameRegex = context.getProperty(ATTRIBUTE_NAME_REGEX).getValue();
         final Pattern attributeNamePattern = attributeNameRegex == null ? null : Pattern.compile(attributeNameRegex);
         final boolean useTransactions = context.getProperty(USE_TRANSACTIONS).asBoolean();
         final String transactionalIdPrefix = context.getProperty(TRANSACTIONAL_ID_PREFIX).evaluateAttributeExpressions().getValue();
-        Supplier<String> transactionalIdSupplier = KafkaProcessorUtils.getTransactionalIdSupplier(transactionalIdPrefix);
+        Supplier<String> transactionalIdSupplier = new TransactionIdSupplier(transactionalIdPrefix);
 
         final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
         final Charset charset = Charset.forName(charsetName);
 
-        final Map<String, Object> kafkaProperties = new HashMap<>();
-        KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
+        final KafkaPropertyProvider propertyProvider = new StandardKafkaPropertyProvider(ProducerConfig.class);
+        final Map<String, Object> kafkaProperties = propertyProvider.getProperties(context);
         kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize));
@@ -407,8 +413,8 @@ public class PublishKafka_1_0 extends AbstractProcessor {
             return;
         }
 
-        final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue();
-        final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
+        final String securityProtocol = context.getProperty(SECURITY_PROTOCOL).getValue();
+        final String bootstrapServers = context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
         final boolean useTransactions = context.getProperty(USE_TRANSACTIONS).asBoolean();
 
         final long startTime = System.nanoTime();
@@ -441,12 +447,9 @@ public class PublishKafka_1_0 extends AbstractProcessor {
                 }
 
                 final Integer partition = getPartition(context, flowFile);
-                session.read(flowFile, new InputStreamCallback() {
-                    @Override
-                    public void process(final InputStream rawIn) throws IOException {
-                        try (final InputStream in = new BufferedInputStream(rawIn)) {
-                            lease.publish(flowFile, in, messageKey, demarcatorBytes, topic, partition);
-                        }
+                session.read(flowFile, rawIn -> {
+                    try (final InputStream in = new BufferedInputStream(rawIn)) {
+                        lease.publish(flowFile, in, messageKey, demarcatorBytes, topic, partition);
                     }
                 });
             }
@@ -469,7 +472,7 @@ public class PublishKafka_1_0 extends AbstractProcessor {
                 success = session.putAttribute(success, MSG_COUNT, String.valueOf(msgCount));
                 session.adjustCounter("Messages Sent", msgCount, true);
 
-                final String transitUri = KafkaProcessorUtils.buildTransitURI(securityProtocol, bootstrapServers, topic);
+                final String transitUri = StandardTransitUriProvider.getTransitUri(securityProtocol, bootstrapServers, topic);
                 session.getProvenanceReporter().send(success, transitUri, "Sent " + msgCount + " messages", transmissionMillis);
                 session.transfer(success, REL_SUCCESS);
             }
@@ -483,7 +486,7 @@ public class PublishKafka_1_0 extends AbstractProcessor {
         if (context.getProperty(KEY).isSet()) {
             uninterpretedKey = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
         } else {
-            uninterpretedKey = flowFile.getAttribute(KafkaProcessorUtils.KAFKA_KEY);
+            uninterpretedKey = flowFile.getAttribute(KafkaFlowFileAttribute.KAFKA_KEY);
         }
 
         if (uninterpretedKey == null) {
@@ -491,7 +494,7 @@ public class PublishKafka_1_0 extends AbstractProcessor {
         }
 
         final String keyEncoding = context.getProperty(KEY_ATTRIBUTE_ENCODING).getValue();
-        if (UTF8_ENCODING.getValue().equals(keyEncoding)) {
+        if (KeyEncoding.UTF8.getValue().equals(keyEncoding)) {
             return uninterpretedKey.getBytes(StandardCharsets.UTF_8);
         }
 
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/record/sink/kafka/KafkaRecordSink_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/record/sink/kafka/KafkaRecordSink_1_0.java
index 287b9aa449..a7d602baa3 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/record/sink/kafka/KafkaRecordSink_1_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/record/sink/kafka/KafkaRecordSink_1_0.java
@@ -16,14 +16,11 @@
  */
 package org.apache.nifi.record.sink.kafka;
 
-import org.apache.commons.lang3.StringUtils;
-import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.config.SaslConfigs;
-import org.apache.kafka.common.config.SslConfigs;
 import org.apache.kafka.common.serialization.ByteArraySerializer;
 import org.apache.nifi.annotation.behavior.DynamicProperty;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
@@ -39,10 +36,13 @@ import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ControllerServiceInitializationContext;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.kerberos.KerberosCredentialsService;
+import org.apache.nifi.kafka.shared.component.KafkaPublishComponent;
+import org.apache.nifi.kafka.shared.property.provider.KafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.property.provider.StandardKafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.validation.DynamicPropertyValidator;
+import org.apache.nifi.kafka.shared.validation.KafkaClientCustomValidationFunction;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils;
 import org.apache.nifi.record.sink.RecordSinkService;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.serialization.RecordSetWriter;
@@ -51,24 +51,19 @@ import org.apache.nifi.serialization.WriteResult;
 import org.apache.nifi.serialization.record.Record;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.RecordSet;
-import org.apache.nifi.ssl.SSLContextService;
 import org.apache.nifi.stream.io.ByteCountingOutputStream;
 import org.apache.nifi.stream.io.exception.TokenTooLargeException;
-import org.apache.nifi.util.FormatUtils;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
-
 @Tags({"kafka", "record", "sink"})
 @CapabilityDescription("Provides a service to write records to a Kafka 1.x topic.")
 @DynamicProperty(name = "The name of a Kafka configuration property.", value = "The value of a given Kafka configuration property.",
@@ -76,7 +71,7 @@ import java.util.concurrent.TimeoutException;
                 + " In the event a dynamic property represents a property that was already set, its value will be ignored and WARN message logged."
                 + " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration. ",
         expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY)
-public class KafkaRecordSink_1_0 extends AbstractControllerService implements RecordSinkService {
+public class KafkaRecordSink_1_0 extends AbstractControllerService implements RecordSinkService, KafkaPublishComponent {
 
     static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("all", "Guarantee Replicated Delivery",
             "Records are considered 'transmitted unsuccessfully' unless the message is replicated to the appropriate "
@@ -89,10 +84,6 @@ public class KafkaRecordSink_1_0 extends AbstractControllerService implements Re
             "Records are considered 'transmitted successfully' after successfully writing the content to a Kafka node, "
                     + "without waiting for a response. This provides the best performance but may result in data loss.");
 
-    static final AllowableValue UTF8_ENCODING = new AllowableValue("utf-8", "UTF-8 Encoded", "The key is interpreted as a UTF-8 Encoded string.");
-    static final AllowableValue HEX_ENCODING = new AllowableValue("hex", "Hex Encoded",
-            "The key is interpreted as arbitrary binary data that is encoded using hexadecimal characters with uppercase letters.");
-
     static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder()
             .name("topic")
             .displayName("Topic Name")
@@ -171,17 +162,18 @@ public class KafkaRecordSink_1_0 extends AbstractControllerService implements Re
     private volatile Producer<byte[], byte[]> producer;
 
     @Override
-    protected void init(final ControllerServiceInitializationContext context) throws InitializationException {
+    protected void init(final ControllerServiceInitializationContext context) {
         final List<PropertyDescriptor> properties = new ArrayList<>();
-        properties.add(KafkaProcessorUtils.BOOTSTRAP_SERVERS);
+        properties.add(BOOTSTRAP_SERVERS);
         properties.add(TOPIC);
         properties.add(RecordSinkService.RECORD_WRITER_FACTORY);
         properties.add(DELIVERY_GUARANTEE);
         properties.add(MESSAGE_HEADER_ENCODING);
-        properties.add(KafkaProcessorUtils.SECURITY_PROTOCOL);
-        properties.add(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE);
-        properties.add(KafkaProcessorUtils.JAAS_SERVICE_NAME);
-        properties.add(KafkaProcessorUtils.SSL_CONTEXT_SERVICE);
+        properties.add(SECURITY_PROTOCOL);
+        properties.add(SASL_MECHANISM);
+        properties.add(KERBEROS_CREDENTIALS_SERVICE);
+        properties.add(KERBEROS_SERVICE_NAME);
+        properties.add(SSL_CONTEXT_SERVICE);
         properties.add(MAX_REQUEST_SIZE);
         properties.add(ACK_WAIT_TIME);
         properties.add(METADATA_WAIT_TIME);
@@ -199,7 +191,7 @@ public class KafkaRecordSink_1_0 extends AbstractControllerService implements Re
         return new PropertyDescriptor.Builder()
                 .description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.")
                 .name(propertyDescriptorName)
-                .addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class))
+                .addValidator(new DynamicPropertyValidator(ProducerConfig.class))
                 .dynamic(true)
                 .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                 .build();
@@ -207,7 +199,7 @@ public class KafkaRecordSink_1_0 extends AbstractControllerService implements Re
 
     @Override
     protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
-        return KafkaProcessorUtils.validateCommonProperties(validationContext);
+        return new KafkaClientCustomValidationFunction().apply(validationContext);
     }
 
     @OnEnabled
@@ -216,12 +208,10 @@ public class KafkaRecordSink_1_0 extends AbstractControllerService implements Re
         writerFactory = context.getProperty(RecordSinkService.RECORD_WRITER_FACTORY).asControllerService(RecordSetWriterFactory.class);
         maxMessageSize = context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue();
         maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
+        maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
 
-        final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
-        final Charset charset = Charset.forName(charsetName);
-
-        final Map<String, Object> kafkaProperties = new HashMap<>();
-        buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
+        final KafkaPropertyProvider kafkaPropertyProvider = new StandardKafkaPropertyProvider(ConsumerConfig.class);
+        final Map<String, Object> kafkaProperties = kafkaPropertyProvider.getProperties(context);
         kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize));
@@ -299,93 +289,12 @@ public class KafkaRecordSink_1_0 extends AbstractControllerService implements Re
     }
 
     @OnDisabled
-    public void stop() throws IOException {
+    public void stop() {
         if (producer != null) {
             producer.close(maxAckWaitMillis, TimeUnit.MILLISECONDS);
         }
     }
 
-    static void buildCommonKafkaProperties(final ConfigurationContext context, final Class<?> kafkaConfigClass, final Map<String, Object> mapToPopulate) {
-        for (PropertyDescriptor propertyDescriptor : context.getProperties().keySet()) {
-            if (propertyDescriptor.equals(KafkaProcessorUtils.SSL_CONTEXT_SERVICE)) {
-                // Translate SSLContext Service configuration into Kafka properties
-                final SSLContextService sslContextService = context.getProperty(KafkaProcessorUtils.SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
-                if (sslContextService != null && sslContextService.isKeyStoreConfigured()) {
-                    mapToPopulate.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, sslContextService.getKeyStoreFile());
-                    mapToPopulate.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, sslContextService.getKeyStorePassword());
-                    final String keyPass = sslContextService.getKeyPassword() == null ? sslContextService.getKeyStorePassword() : sslContextService.getKeyPassword();
-                    mapToPopulate.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, keyPass);
-                    mapToPopulate.put(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, sslContextService.getKeyStoreType());
-                }
-
-                if (sslContextService != null && sslContextService.isTrustStoreConfigured()) {
-                    mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, sslContextService.getTrustStoreFile());
-                    mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, sslContextService.getTrustStorePassword());
-                    mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, sslContextService.getTrustStoreType());
-                }
-            }
-
-            String propertyName = propertyDescriptor.getName();
-            String propertyValue = propertyDescriptor.isExpressionLanguageSupported()
-                    ? context.getProperty(propertyDescriptor).evaluateAttributeExpressions().getValue()
-                    : context.getProperty(propertyDescriptor).getValue();
-
-            if (propertyValue != null) {
-                // If the property name ends in ".ms" then it is a time period. We want to accept either an integer as number of milliseconds
-                // or the standard NiFi time period such as "5 secs"
-                if (propertyName.endsWith(".ms") && !StringUtils.isNumeric(propertyValue.trim())) { // kafka standard time notation
-                    propertyValue = String.valueOf(FormatUtils.getTimeDuration(propertyValue.trim(), TimeUnit.MILLISECONDS));
-                }
-
-                if (KafkaProcessorUtils.isStaticStringFieldNamePresent(propertyName, kafkaConfigClass, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class)) {
-                    mapToPopulate.put(propertyName, propertyValue);
-                }
-            }
-        }
-
-        String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue();
-        if (KafkaProcessorUtils.SEC_SASL_PLAINTEXT.getValue().equals(securityProtocol) || KafkaProcessorUtils.SEC_SASL_SSL.getValue().equals(securityProtocol)) {
-            setJaasConfig(mapToPopulate, context);
-        }
-    }
-
-    /**
-     * Method used to configure the 'sasl.jaas.config' property based on KAFKA-4259<br />
-     * https://cwiki.apache.org/confluence/display/KAFKA/KIP-85%3A+Dynamic+JAAS+configuration+for+Kafka+clients<br />
-     * <br />
-     * It expects something with the following format: <br />
-     * <br />
-     * &lt;LoginModuleClass&gt; &lt;ControlFlag&gt; *(&lt;OptionName&gt;=&lt;OptionValue&gt;); <br />
-     * ControlFlag = required / requisite / sufficient / optional
-     *
-     * @param mapToPopulate Map of configuration properties
-     * @param context       Context
-     */
-    private static void setJaasConfig(Map<String, Object> mapToPopulate, ConfigurationContext context) {
-        String keytab = null;
-        String principal = null;
-
-        // If the Kerberos Credentials Service is specified, we need to use its configuration, not the explicit properties for principal/keytab.
-        // The customValidate method ensures that only one can be set, so we know that the principal & keytab above are null.
-        final KerberosCredentialsService credentialsService = context.getProperty(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
-        if (credentialsService != null) {
-            principal = credentialsService.getPrincipal();
-            keytab = credentialsService.getKeytab();
-        }
-
-
-        String serviceName = context.getProperty(KafkaProcessorUtils.JAAS_SERVICE_NAME).evaluateAttributeExpressions().getValue();
-        if (StringUtils.isNotBlank(keytab) && StringUtils.isNotBlank(principal) && StringUtils.isNotBlank(serviceName)) {
-            mapToPopulate.put(SaslConfigs.SASL_JAAS_CONFIG, "com.sun.security.auth.module.Krb5LoginModule required "
-                    + "useTicketCache=false "
-                    + "renewTicket=true "
-                    + "serviceName=\"" + serviceName + "\" "
-                    + "useKeyTab=true "
-                    + "keyTab=\"" + keytab + "\" "
-                    + "principal=\"" + principal + "\";");
-        }
-    }
-
     // this getter is intended explicitly for testing purposes
     protected RecordSetWriterFactory getWriterFactory() {
         return this.writerFactory;
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java
index ff276b665b..22a7b4ddb0 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java
@@ -18,6 +18,7 @@ package org.apache.nifi.processors.kafka.pubsub;
 
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.nifi.kafka.shared.property.SecurityProtocol;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.jupiter.api.BeforeEach;
@@ -39,80 +40,74 @@ public class ConsumeKafkaTest {
     }
 
     @Test
-    public void validateCustomValidatorSettings() throws Exception {
+    public void validateCustomValidatorSettings() {
         ConsumeKafka_1_0 consumeKafka = new ConsumeKafka_1_0();
         TestRunner runner = TestRunners.newTestRunner(consumeKafka);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
+        runner.setProperty(ConsumeKafka_1_0.BOOTSTRAP_SERVERS, "okeydokey:1234");
         runner.setProperty(ConsumeKafka_1_0.TOPICS, "foo");
         runner.setProperty(ConsumeKafka_1_0.GROUP_ID, "foo");
         runner.setProperty(ConsumeKafka_1_0.AUTO_OFFSET_RESET, ConsumeKafka_1_0.OFFSET_EARLIEST);
         runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
         runner.assertValid();
-        runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "Foo");
-        runner.assertNotValid();
-        runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
-        runner.assertValid();
         runner.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
         runner.assertValid();
-        runner.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true");
-        runner.assertNotValid();
     }
 
     @Test
-    public void validatePropertiesValidation() throws Exception {
+    public void validatePropertiesValidation() {
         ConsumeKafka_1_0 consumeKafka = new ConsumeKafka_1_0();
         TestRunner runner = TestRunners.newTestRunner(consumeKafka);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
+        runner.setProperty(ConsumeKafka_1_0.BOOTSTRAP_SERVERS, "okeydokey:1234");
         runner.setProperty(ConsumeKafka_1_0.TOPICS, "foo");
         runner.setProperty(ConsumeKafka_1_0.GROUP_ID, "foo");
         runner.setProperty(ConsumeKafka_1_0.AUTO_OFFSET_RESET, ConsumeKafka_1_0.OFFSET_EARLIEST);
 
         runner.removeProperty(ConsumeKafka_1_0.GROUP_ID);
 
-        AssertionError e = assertThrows(AssertionError.class, () -> runner.assertValid());
+        AssertionError e = assertThrows(AssertionError.class, runner::assertValid);
         assertTrue(e.getMessage().contains("invalid because Group ID is required"));
 
         runner.setProperty(ConsumeKafka_1_0.GROUP_ID, "");
 
-        e = assertThrows(AssertionError.class, () -> runner.assertValid());
+        e = assertThrows(AssertionError.class, runner::assertValid);
         assertTrue(e.getMessage().contains("must contain at least one character that is not white space"));
 
         runner.setProperty(ConsumeKafka_1_0.GROUP_ID, "  ");
 
-        e = assertThrows(AssertionError.class, () -> runner.assertValid());
+        e = assertThrows(AssertionError.class, runner::assertValid);
         assertTrue(e.getMessage().contains("must contain at least one character that is not white space"));
     }
 
     @Test
-    public void testJaasConfiguration() throws Exception {
+    public void testJaasConfiguration() {
         ConsumeKafka_1_0 consumeKafka = new ConsumeKafka_1_0();
         TestRunner runner = TestRunners.newTestRunner(consumeKafka);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
+        runner.setProperty(ConsumeKafka_1_0.BOOTSTRAP_SERVERS, "okeydokey:1234");
         runner.setProperty(ConsumeKafka_1_0.TOPICS, "foo");
         runner.setProperty(ConsumeKafka_1_0.GROUP_ID, "foo");
         runner.setProperty(ConsumeKafka_1_0.AUTO_OFFSET_RESET, ConsumeKafka_1_0.OFFSET_EARLIEST);
 
-        runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT);
+        runner.setProperty(ConsumeKafka_1_0.SECURITY_PROTOCOL, SecurityProtocol.SASL_PLAINTEXT.name());
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.JAAS_SERVICE_NAME, "kafka");
-        runner.assertValid();
+        runner.setProperty(ConsumeKafka_1_0.KERBEROS_SERVICE_NAME, "kafka");
+        runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.USER_PRINCIPAL, "nifi@APACHE.COM");
+        runner.setProperty(ConsumeKafka_1_0.KERBEROS_PRINCIPAL, "nifi@APACHE.COM");
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "not.A.File");
+        runner.setProperty(ConsumeKafka_1_0.KERBEROS_KEYTAB, "not.A.File");
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "src/test/resources/server.properties");
+        runner.setProperty(ConsumeKafka_1_0.KERBEROS_KEYTAB, "src/test/resources/server.properties");
         runner.assertValid();
 
         runner.setVariable("keytab", "src/test/resources/server.properties");
         runner.setVariable("principal", "nifi@APACHE.COM");
         runner.setVariable("service", "kafka");
-        runner.setProperty(KafkaProcessorUtils.USER_PRINCIPAL, "${principal}");
-        runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "${keytab}");
-        runner.setProperty(KafkaProcessorUtils.JAAS_SERVICE_NAME, "${service}");
+        runner.setProperty(ConsumeKafka_1_0.KERBEROS_PRINCIPAL, "${principal}");
+        runner.setProperty(ConsumeKafka_1_0.KERBEROS_KEYTAB, "${keytab}");
+        runner.setProperty(ConsumeKafka_1_0.KERBEROS_SERVICE_NAME, "${service}");
         runner.assertValid();
     }
 
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java
index 13a0e78912..1a54d0d13d 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java
@@ -51,8 +51,7 @@ public class ConsumerPoolTest {
 
     private Consumer<byte[], byte[]> consumer = null;
     private ProcessSession mockSession = null;
-    private ProcessContext mockContext = Mockito.mock(ProcessContext.class);
-    private ProvenanceReporter mockReporter = null;
+    private final ProcessContext mockContext = Mockito.mock(ProcessContext.class);
     private ConsumerPool testPool = null;
     private ConsumerPool testDemarcatedPool = null;
     private ComponentLog logger = null;
@@ -63,7 +62,7 @@ public class ConsumerPoolTest {
         consumer = mock(Consumer.class);
         logger = mock(ComponentLog.class);
         mockSession = mock(ProcessSession.class);
-        mockReporter = mock(ProvenanceReporter.class);
+        final ProvenanceReporter mockReporter = mock(ProvenanceReporter.class);
         when(mockSession.getProvenanceReporter()).thenReturn(mockReporter);
         testPool = new ConsumerPool(
                 1,
@@ -104,7 +103,7 @@ public class ConsumerPoolTest {
     }
 
     @Test
-    public void validatePoolSimpleCreateClose() throws Exception {
+    public void validatePoolSimpleCreateClose() {
 
         when(consumer.poll(anyLong())).thenReturn(createConsumerRecords("nifi", 0, 0L, new byte[][]{}));
         try (final ConsumerLease lease = testPool.obtainConsumer(mockSession, mockContext)) {
@@ -130,7 +129,7 @@ public class ConsumerPoolTest {
 
     @Test
     @SuppressWarnings("unchecked")
-    public void validatePoolSimpleCreatePollClose() throws Exception {
+    public void validatePoolSimpleCreatePollClose() {
         final byte[][] firstPassValues = new byte[][]{
             "Hello-1".getBytes(StandardCharsets.UTF_8),
             "Hello-2".getBytes(StandardCharsets.UTF_8),
@@ -153,7 +152,7 @@ public class ConsumerPoolTest {
     }
 
     @Test
-    public void validatePoolSimpleBatchCreateClose() throws Exception {
+    public void validatePoolSimpleBatchCreateClose() {
         when(consumer.poll(anyLong())).thenReturn(createConsumerRecords("nifi", 0, 0L, new byte[][]{}));
         for (int i = 0; i < 100; i++) {
             try (final ConsumerLease lease = testPool.obtainConsumer(mockSession, mockContext)) {
@@ -173,7 +172,7 @@ public class ConsumerPoolTest {
 
     @Test
     @SuppressWarnings("unchecked")
-    public void validatePoolBatchCreatePollClose() throws Exception {
+    public void validatePoolBatchCreatePollClose() {
         final byte[][] firstPassValues = new byte[][]{
             "Hello-1".getBytes(StandardCharsets.UTF_8),
             "Hello-2".getBytes(StandardCharsets.UTF_8),
@@ -200,7 +199,7 @@ public class ConsumerPoolTest {
 
         when(consumer.poll(anyLong())).thenThrow(new KafkaException("oops"));
         try (final ConsumerLease lease = testPool.obtainConsumer(mockSession, mockContext)) {
-            assertThrows(KafkaException.class, () -> lease.poll());
+            assertThrows(KafkaException.class, lease::poll);
         }
         testPool.close();
         verify(mockSession, times(0)).create();
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ITConsumeKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ITConsumeKafka.java
index a7eb913344..a5f33db041 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ITConsumeKafka.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ITConsumeKafka.java
@@ -42,7 +42,7 @@ public class ITConsumeKafka {
     }
 
     @Test
-    public void validateGetAllMessages() throws Exception {
+    public void validateGetAllMessages() {
         String groupName = "validateGetAllMessages";
 
         when(mockConsumerPool.obtainConsumer(any(), any())).thenReturn(mockLease);
@@ -56,7 +56,7 @@ public class ITConsumeKafka {
             }
         };
         final TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
+        runner.setProperty(ConsumeKafka_1_0.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
         runner.setProperty(ConsumeKafka_1_0.TOPICS, "foo,bar");
         runner.setProperty(ConsumeKafka_1_0.GROUP_ID, groupName);
         runner.setProperty(ConsumeKafka_1_0.AUTO_OFFSET_RESET, ConsumeKafka_1_0.OFFSET_EARLIEST);
@@ -72,7 +72,7 @@ public class ITConsumeKafka {
     }
 
     @Test
-    public void validateGetAllMessagesPattern() throws Exception {
+    public void validateGetAllMessagesPattern() {
         String groupName = "validateGetAllMessagesPattern";
 
         when(mockConsumerPool.obtainConsumer(any(), any())).thenReturn(mockLease);
@@ -86,7 +86,7 @@ public class ITConsumeKafka {
             }
         };
         final TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
+        runner.setProperty(ConsumeKafka_1_0.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
         runner.setProperty(ConsumeKafka_1_0.TOPICS, "(fo.*)|(ba)");
         runner.setProperty(ConsumeKafka_1_0.TOPIC_TYPE, "pattern");
         runner.setProperty(ConsumeKafka_1_0.GROUP_ID, groupName);
@@ -103,7 +103,7 @@ public class ITConsumeKafka {
     }
 
     @Test
-    public void validateGetErrorMessages() throws Exception {
+    public void validateGetErrorMessages() {
         String groupName = "validateGetErrorMessages";
 
         when(mockConsumerPool.obtainConsumer(any(), any())).thenReturn(mockLease);
@@ -117,7 +117,7 @@ public class ITConsumeKafka {
             }
         };
         final TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
+        runner.setProperty(ConsumeKafka_1_0.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
         runner.setProperty(ConsumeKafka_1_0.TOPICS, "foo,bar");
         runner.setProperty(ConsumeKafka_1_0.GROUP_ID, groupName);
         runner.setProperty(ConsumeKafka_1_0.AUTO_OFFSET_RESET, ConsumeKafka_1_0.OFFSET_EARLIEST);
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumeKafkaRecord_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumeKafkaRecord_1_0.java
index 439c2f3fa5..d2dae0f04c 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumeKafkaRecord_1_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumeKafkaRecord_1_0.java
@@ -18,6 +18,7 @@ package org.apache.nifi.processors.kafka.pubsub;
 
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.nifi.kafka.shared.property.SecurityProtocol;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processors.kafka.pubsub.util.MockRecordParser;
@@ -58,7 +59,7 @@ public class TestConsumeKafkaRecord_1_0 {
         };
 
         runner = TestRunners.newTestRunner(proc);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
+        runner.setProperty(ConsumeKafkaRecord_1_0.BOOTSTRAP_SERVERS, "okeydokey:1234");
 
         final String readerId = "record-reader";
         final MockRecordParser readerService = new MockRecordParser();
@@ -77,24 +78,18 @@ public class TestConsumeKafkaRecord_1_0 {
     }
 
     @Test
-    public void validateCustomValidatorSettings() throws Exception {
+    public void validateCustomValidatorSettings() {
         runner.setProperty(ConsumeKafkaRecord_1_0.TOPICS, "foo");
         runner.setProperty(ConsumeKafkaRecord_1_0.GROUP_ID, "foo");
         runner.setProperty(ConsumeKafkaRecord_1_0.AUTO_OFFSET_RESET, ConsumeKafkaRecord_1_0.OFFSET_EARLIEST);
         runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
         runner.assertValid();
-        runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "Foo");
-        runner.assertNotValid();
-        runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
-        runner.assertValid();
         runner.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
         runner.assertValid();
-        runner.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true");
-        runner.assertNotValid();
     }
 
     @Test
-    public void validatePropertiesValidation() throws Exception {
+    public void validatePropertiesValidation() {
         runner.setProperty(ConsumeKafkaRecord_1_0.TOPICS, "foo");
         runner.setProperty(ConsumeKafkaRecord_1_0.GROUP_ID, "foo");
         runner.setProperty(ConsumeKafkaRecord_1_0.AUTO_OFFSET_RESET, ConsumeKafkaRecord_1_0.OFFSET_EARLIEST);
@@ -115,7 +110,7 @@ public class TestConsumeKafkaRecord_1_0 {
     }
 
     @Test
-    public void validateGetAllMessages() throws Exception {
+    public void validateGetAllMessages() {
         String groupName = "validateGetAllMessages";
 
         when(mockConsumerPool.obtainConsumer(any(), any())).thenReturn(mockLease);
@@ -137,7 +132,7 @@ public class TestConsumeKafkaRecord_1_0 {
     }
 
     @Test
-    public void validateGetAllMessagesPattern() throws Exception {
+    public void validateGetAllMessagesPattern() {
         String groupName = "validateGetAllMessagesPattern";
 
         when(mockConsumerPool.obtainConsumer(any(), any())).thenReturn(mockLease);
@@ -160,7 +155,7 @@ public class TestConsumeKafkaRecord_1_0 {
     }
 
     @Test
-    public void validateGetErrorMessages() throws Exception {
+    public void validateGetErrorMessages() {
         String groupName = "validateGetErrorMessages";
 
         when(mockConsumerPool.obtainConsumer(any(), any())).thenReturn(mockLease);
@@ -182,24 +177,24 @@ public class TestConsumeKafkaRecord_1_0 {
     }
 
     @Test
-    public void testJaasConfiguration() throws Exception {
+    public void testJaasConfiguration() {
         runner.setProperty(ConsumeKafkaRecord_1_0.TOPICS, "foo");
         runner.setProperty(ConsumeKafkaRecord_1_0.GROUP_ID, "foo");
         runner.setProperty(ConsumeKafkaRecord_1_0.AUTO_OFFSET_RESET, ConsumeKafkaRecord_1_0.OFFSET_EARLIEST);
 
-        runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT);
+        runner.setProperty(ConsumeKafkaRecord_1_0.SECURITY_PROTOCOL, SecurityProtocol.SASL_PLAINTEXT.name());
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.JAAS_SERVICE_NAME, "kafka");
-        runner.assertValid();
+        runner.setProperty(ConsumeKafkaRecord_1_0.KERBEROS_SERVICE_NAME, "kafka");
+        runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.USER_PRINCIPAL, "nifi@APACHE.COM");
+        runner.setProperty(ConsumeKafkaRecord_1_0.KERBEROS_PRINCIPAL, "nifi@APACHE.COM");
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "not.A.File");
+        runner.setProperty(ConsumeKafkaRecord_1_0.KERBEROS_KEYTAB, "not.A.File");
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "src/test/resources/server.properties");
+        runner.setProperty(ConsumeKafkaRecord_1_0.KERBEROS_KEYTAB, "src/test/resources/server.properties");
         runner.assertValid();
     }
 
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/record/sink/kafka/TestKafkaRecordSink_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/record/sink/kafka/TestKafkaRecordSink_1_0.java
index ae0da43069..eb42d34019 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/record/sink/kafka/TestKafkaRecordSink_1_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/record/sink/kafka/TestKafkaRecordSink_1_0.java
@@ -27,10 +27,10 @@ import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ControllerServiceInitializationContext;
+import org.apache.nifi.kafka.shared.property.SecurityProtocol;
 import org.apache.nifi.kerberos.KerberosCredentialsService;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils;
 import org.apache.nifi.record.sink.RecordSinkService;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.serialization.RecordSetWriterFactory;
@@ -138,13 +138,13 @@ public class TestKafkaRecordSink_1_0 {
         when(context.getProperty(KafkaRecordSink_1_0.MESSAGE_HEADER_ENCODING)).thenReturn(charEncodingValue);
 
         final PropertyValue securityValue = Mockito.mock(StandardPropertyValue.class);
-        when(securityValue.getValue()).thenReturn(KafkaProcessorUtils.SEC_SASL_PLAINTEXT.getValue());
-        when(context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL)).thenReturn(securityValue);
+        when(securityValue.getValue()).thenReturn(SecurityProtocol.PLAINTEXT.name());
+        when(context.getProperty(KafkaRecordSink_1_0.SECURITY_PROTOCOL)).thenReturn(securityValue);
 
         final PropertyValue jaasValue = Mockito.mock(StandardPropertyValue.class);
         when(jaasValue.evaluateAttributeExpressions()).thenReturn(jaasValue);
         when(jaasValue.getValue()).thenReturn(null);
-        when(context.getProperty(KafkaProcessorUtils.JAAS_SERVICE_NAME)).thenReturn(jaasValue);
+        when(context.getProperty(KafkaRecordSink_1_0.KERBEROS_SERVICE_NAME)).thenReturn(jaasValue);
 
         Map<PropertyDescriptor, String> propertyMap = new HashMap<>();
         propertyMap.put(KafkaRecordSink_1_0.TOPIC, KafkaRecordSink_1_0.TOPIC.getName());
@@ -160,9 +160,9 @@ public class TestKafkaRecordSink_1_0 {
         MockRecordWriter writer = new MockRecordWriter(null, false);
         when(context.getProperty(RecordSinkService.RECORD_WRITER_FACTORY)).thenReturn(pValue);
         when(pValue.asControllerService(RecordSetWriterFactory.class)).thenReturn(writer);
-        when(context.getProperty(KafkaProcessorUtils.SSL_CONTEXT_SERVICE)).thenReturn(pValue);
+        when(context.getProperty(KafkaRecordSink_1_0.SSL_CONTEXT_SERVICE)).thenReturn(pValue);
         when(pValue.asControllerService(SSLContextService.class)).thenReturn(null);
-        when(context.getProperty(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE)).thenReturn(pValue);
+        when(context.getProperty(KafkaRecordSink_1_0.KERBEROS_CREDENTIALS_SERVICE)).thenReturn(pValue);
         when(pValue.asControllerService(KerberosCredentialsService.class)).thenReturn(null);
 
         final ControllerServiceInitializationContext initContext = new MockControllerServiceInitializationContext(task, UUID.randomUUID().toString(), logger, stateManager);
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/resources/log4j.properties b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/resources/log4j.properties
deleted file mode 100644
index 57cd63f139..0000000000
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/resources/log4j.properties
+++ /dev/null
@@ -1,21 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#    http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-log4j.rootCategory=INFO, stdout
-
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=%d{ABSOLUTE} %5p %t %c{2}:%L - %m%n
-
-#og4j.category.org.apache.nifi.processors.kafka=DEBUG
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/resources/zookeeper.properties b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/resources/zookeeper.properties
deleted file mode 100644
index f5c257efeb..0000000000
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/resources/zookeeper.properties
+++ /dev/null
@@ -1,20 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#    http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-# the directory where the snapshot is stored.
-dataDir=target/kafka-tmp/zookeeper
-# the port at which the clients will connect
-#clientPort=2181
-# disable the per-ip limit on the number of connections since this is a non-production config
-maxClientCnxns=0
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 3aeedc6a6c..5fd2dc2381 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
@@ -52,7 +52,19 @@
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-kerberos-credentials-service-api</artifactId>
         </dependency>
-
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-kerberos-user-service-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-kerberos</artifactId>
+            <version>1.19.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-kafka-shared</artifactId>
+        </dependency>
         <dependency>
             <groupId>org.apache.kafka</groupId>
             <artifactId>kafka-clients</artifactId>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_2_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_2_0.java
index de1ffee237..e0e0cb5894 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_2_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_2_0.java
@@ -34,6 +34,13 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyDescriptor.Builder;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute;
+import org.apache.nifi.kafka.shared.component.KafkaClientComponent;
+import org.apache.nifi.kafka.shared.property.KeyEncoding;
+import org.apache.nifi.kafka.shared.property.provider.KafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.property.provider.StandardKafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.validation.DynamicPropertyValidator;
+import org.apache.nifi.kafka.shared.validation.KafkaClientCustomValidationFunction;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -49,7 +56,6 @@ import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -59,9 +65,6 @@ import java.util.regex.Pattern;
 
 import static org.apache.nifi.expression.ExpressionLanguageScope.NONE;
 import static org.apache.nifi.expression.ExpressionLanguageScope.VARIABLE_REGISTRY;
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.DO_NOT_ADD_KEY_AS_ATTRIBUTE;
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING;
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING;
 
 @CapabilityDescription("Consumes messages from Apache Kafka specifically built against the Kafka 2.0 Consumer API. "
     + "The complementary NiFi processor for sending messages is PublishKafkaRecord_2_0. Please note that, at this time, the Processor assumes that "
@@ -74,9 +77,9 @@ import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_E
 @WritesAttributes({
     @WritesAttribute(attribute = "record.count", description = "The number of records received"),
     @WritesAttribute(attribute = "mime.type", description = "The MIME Type that is provided by the configured Record Writer"),
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_PARTITION, description = "The partition of the topic the records are from"),
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_TIMESTAMP, description = "The timestamp of the message in the partition of the topic."),
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_TOPIC, description = "The topic records are from")
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_PARTITION, description = "The partition of the topic the records are from"),
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_TIMESTAMP, description = "The timestamp of the message in the partition of the topic."),
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_TOPIC, description = "The topic records are from")
 })
 @InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
 @DynamicProperty(name = "The name of a Kafka configuration property.", value = "The value of a given Kafka configuration property.",
@@ -85,7 +88,7 @@ import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_E
         + " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration.",
         expressionLanguageScope = VARIABLE_REGISTRY)
 @SeeAlso({ConsumeKafka_2_0.class, PublishKafka_2_0.class, PublishKafkaRecord_2_0.class})
-public class ConsumeKafkaRecord_2_0 extends AbstractProcessor {
+public class ConsumeKafkaRecord_2_0 extends AbstractProcessor implements KafkaClientComponent {
 
     static final AllowableValue OFFSET_EARLIEST = new AllowableValue("earliest", "earliest", "Automatically reset the offset to the earliest offset");
     static final AllowableValue OFFSET_LATEST = new AllowableValue("latest", "latest", "Automatically reset the offset to the latest offset");
@@ -223,11 +226,11 @@ public class ConsumeKafkaRecord_2_0 extends AbstractProcessor {
     static final PropertyDescriptor KEY_ATTRIBUTE_ENCODING = new PropertyDescriptor.Builder()
         .name("key-attribute-encoding")
         .displayName("Key Attribute Encoding")
-        .description("If the <Separate By Key> property is set to true, FlowFiles that are emitted have an attribute named '" + KafkaProcessorUtils.KAFKA_KEY +
+        .description("If the <Separate By Key> property is set to true, FlowFiles that are emitted have an attribute named '" + KafkaFlowFileAttribute.KAFKA_KEY +
             "'. This property dictates how the value of the attribute should be encoded.")
         .required(true)
-        .defaultValue(UTF8_ENCODING.getValue())
-        .allowableValues(UTF8_ENCODING, HEX_ENCODING, DO_NOT_ADD_KEY_AS_ATTRIBUTE)
+        .defaultValue(KeyEncoding.UTF8.getValue())
+        .allowableValues(KeyEncoding.class)
         .build();
 
     static final Relationship REL_SUCCESS = new Relationship.Builder()
@@ -248,22 +251,22 @@ public class ConsumeKafkaRecord_2_0 extends AbstractProcessor {
 
     static {
         List<PropertyDescriptor> descriptors = new ArrayList<>();
-        descriptors.add(KafkaProcessorUtils.BOOTSTRAP_SERVERS);
+        descriptors.add(BOOTSTRAP_SERVERS);
         descriptors.add(TOPICS);
         descriptors.add(TOPIC_TYPE);
         descriptors.add(RECORD_READER);
         descriptors.add(RECORD_WRITER);
         descriptors.add(HONOR_TRANSACTIONS);
-        descriptors.add(KafkaProcessorUtils.SECURITY_PROTOCOL);
-        descriptors.add(KafkaProcessorUtils.SASL_MECHANISM);
-        descriptors.add(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE);
-        descriptors.add(KafkaProcessorUtils.JAAS_SERVICE_NAME);
-        descriptors.add(KafkaProcessorUtils.USER_PRINCIPAL);
-        descriptors.add(KafkaProcessorUtils.USER_KEYTAB);
-        descriptors.add(KafkaProcessorUtils.USERNAME);
-        descriptors.add(KafkaProcessorUtils.PASSWORD);
-        descriptors.add(KafkaProcessorUtils.TOKEN_AUTH);
-        descriptors.add(KafkaProcessorUtils.SSL_CONTEXT_SERVICE);
+        descriptors.add(SECURITY_PROTOCOL);
+        descriptors.add(SASL_MECHANISM);
+        descriptors.add(KERBEROS_CREDENTIALS_SERVICE);
+        descriptors.add(KERBEROS_SERVICE_NAME);
+        descriptors.add(KERBEROS_PRINCIPAL);
+        descriptors.add(KERBEROS_KEYTAB);
+        descriptors.add(SASL_USERNAME);
+        descriptors.add(SASL_PASSWORD);
+        descriptors.add(TOKEN_AUTHENTICATION);
+        descriptors.add(SSL_CONTEXT_SERVICE);
         descriptors.add(GROUP_ID);
         descriptors.add(SEPARATE_BY_KEY);
         descriptors.add(KEY_ATTRIBUTE_ENCODING);
@@ -306,7 +309,7 @@ public class ConsumeKafkaRecord_2_0 extends AbstractProcessor {
         return new Builder()
                 .description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.")
                 .name(propertyDescriptorName)
-                .addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ConsumerConfig.class))
+                .addValidator(new DynamicPropertyValidator(ConsumerConfig.class))
                 .dynamic(true)
                 .expressionLanguageSupported(VARIABLE_REGISTRY)
                 .build();
@@ -314,7 +317,7 @@ public class ConsumeKafkaRecord_2_0 extends AbstractProcessor {
 
     @Override
     protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
-        final Collection<ValidationResult> validationResults = KafkaProcessorUtils.validateCommonProperties(validationContext);
+        final Collection<ValidationResult> validationResults = new KafkaClientCustomValidationFunction().apply(validationContext);
 
         final ValidationResult consumerPartitionsResult = ConsumerPartitionsUtil.validateConsumePartitions(validationContext.getAllProperties());
         validationResults.add(consumerPartitionsResult);
@@ -368,16 +371,16 @@ public class ConsumeKafkaRecord_2_0 extends AbstractProcessor {
         final int maxLeases = context.getMaxConcurrentTasks();
         final long maxUncommittedTime = context.getProperty(MAX_UNCOMMITTED_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
 
-        final Map<String, Object> props = new HashMap<>();
-        KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props);
-        props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
+        final KafkaPropertyProvider propertyProvider = new StandardKafkaPropertyProvider(ConsumerConfig.class);
+        final Map<String, Object> props = propertyProvider.getProperties(context);
+        props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, Boolean.FALSE.toString());
         props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
         props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
         final String topicListing = context.getProperty(ConsumeKafkaRecord_2_0.TOPICS).evaluateAttributeExpressions().getValue();
         final String topicType = context.getProperty(ConsumeKafkaRecord_2_0.TOPIC_TYPE).evaluateAttributeExpressions().getValue();
         final List<String> topics = new ArrayList<>();
-        final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue();
-        final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
+        final String securityProtocol = context.getProperty(SECURITY_PROTOCOL).getValue();
+        final String bootstrapServers = context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
 
         final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
         final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_2_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_2_0.java
index 4a6fd1b171..e28852cdc9 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_2_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_2_0.java
@@ -34,6 +34,13 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute;
+import org.apache.nifi.kafka.shared.component.KafkaClientComponent;
+import org.apache.nifi.kafka.shared.property.KeyEncoding;
+import org.apache.nifi.kafka.shared.property.provider.KafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.property.provider.StandardKafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.validation.DynamicPropertyValidator;
+import org.apache.nifi.kafka.shared.validation.KafkaClientCustomValidationFunction;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -48,7 +55,6 @@ import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -56,20 +62,17 @@ import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING;
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING;
-
 @CapabilityDescription("Consumes messages from Apache Kafka specifically built against the Kafka 2.0 Consumer API. "
     + "The complementary NiFi processor for sending messages is PublishKafka_2_0.")
 @Tags({"Kafka", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume", "2.0"})
 @WritesAttributes({
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_COUNT, description = "The number of messages written if more than one"),
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_KEY, description = "The key of message if present and if single message. "
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_COUNT, description = "The number of messages written if more than one"),
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_KEY, description = "The key of message if present and if single message. "
             + "How the key is encoded depends on the value of the 'Key Attribute Encoding' property."),
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_OFFSET, description = "The offset of the message in the partition of the topic."),
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_TIMESTAMP, description = "The timestamp of the message in the partition of the topic."),
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_PARTITION, description = "The partition of the topic the message or message bundle is from"),
-    @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_TOPIC, description = "The topic the message or message bundle is from")
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_OFFSET, description = "The offset of the message in the partition of the topic."),
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_TIMESTAMP, description = "The timestamp of the message in the partition of the topic."),
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_PARTITION, description = "The partition of the topic the message or message bundle is from"),
+    @WritesAttribute(attribute = KafkaFlowFileAttribute.KAFKA_TOPIC, description = "The topic the message or message bundle is from")
 })
 @InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
 @DynamicProperty(name = "The name of a Kafka configuration property.", value = "The value of a given Kafka configuration property.",
@@ -77,7 +80,7 @@ import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_E
         + " In the event a dynamic property represents a property that was already set, its value will be ignored and WARN message logged."
         + " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration. ",
         expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY)
-public class ConsumeKafka_2_0 extends AbstractProcessor {
+public class ConsumeKafka_2_0 extends AbstractProcessor implements KafkaClientComponent {
     static final AllowableValue OFFSET_EARLIEST = new AllowableValue("earliest", "earliest", "Automatically reset the offset to the earliest offset");
 
     static final AllowableValue OFFSET_LATEST = new AllowableValue("latest", "latest", "Automatically reset the offset to the latest offset");
@@ -128,10 +131,10 @@ public class ConsumeKafka_2_0 extends AbstractProcessor {
     static final PropertyDescriptor KEY_ATTRIBUTE_ENCODING = new PropertyDescriptor.Builder()
             .name("key-attribute-encoding")
             .displayName("Key Attribute Encoding")
-            .description("FlowFiles that are emitted have an attribute named '" + KafkaProcessorUtils.KAFKA_KEY + "'. This property dictates how the value of the attribute should be encoded.")
+            .description("FlowFiles that are emitted have an attribute named '" + KafkaFlowFileAttribute.KAFKA_KEY + "'. This property dictates how the value of the attribute should be encoded.")
             .required(true)
-            .defaultValue(UTF8_ENCODING.getValue())
-            .allowableValues(UTF8_ENCODING, HEX_ENCODING)
+            .defaultValue(KeyEncoding.UTF8.getValue())
+            .allowableValues(KeyEncoding.class)
             .build();
 
     static final PropertyDescriptor MESSAGE_DEMARCATOR = new PropertyDescriptor.Builder()
@@ -236,7 +239,17 @@ public class ConsumeKafka_2_0 extends AbstractProcessor {
 
     static {
         List<PropertyDescriptor> descriptors = new ArrayList<>();
-        descriptors.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors());
+        descriptors.add(BOOTSTRAP_SERVERS);
+        descriptors.add(SECURITY_PROTOCOL);
+        descriptors.add(SASL_MECHANISM);
+        descriptors.add(KERBEROS_SERVICE_NAME);
+        descriptors.add(KERBEROS_CREDENTIALS_SERVICE);
+        descriptors.add(KERBEROS_PRINCIPAL);
+        descriptors.add(KERBEROS_KEYTAB);
+        descriptors.add(SASL_USERNAME);
+        descriptors.add(SASL_PASSWORD);
+        descriptors.add(TOKEN_AUTHENTICATION);
+        descriptors.add(SSL_CONTEXT_SERVICE);
         descriptors.add(TOPICS);
         descriptors.add(TOPIC_TYPE);
         descriptors.add(HONOR_TRANSACTIONS);
@@ -278,7 +291,7 @@ public class ConsumeKafka_2_0 extends AbstractProcessor {
         return new PropertyDescriptor.Builder()
                 .description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.")
                 .name(propertyDescriptorName)
-                .addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ConsumerConfig.class))
+                .addValidator(new DynamicPropertyValidator(ConsumerConfig.class))
                 .dynamic(true)
                 .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                 .build();
@@ -286,7 +299,7 @@ public class ConsumeKafka_2_0 extends AbstractProcessor {
 
     @Override
     protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
-        final Collection<ValidationResult> validationResults = KafkaProcessorUtils.validateCommonProperties(validationContext);
+        final Collection<ValidationResult> validationResults = new KafkaClientCustomValidationFunction().apply(validationContext);
 
         final ValidationResult consumerPartitionsResult = ConsumerPartitionsUtil.validateConsumePartitions(validationContext.getAllProperties());
         validationResults.add(consumerPartitionsResult);
@@ -342,9 +355,9 @@ public class ConsumeKafka_2_0 extends AbstractProcessor {
         final byte[] demarcator = context.getProperty(ConsumeKafka_2_0.MESSAGE_DEMARCATOR).isSet()
                 ? context.getProperty(ConsumeKafka_2_0.MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8)
                 : null;
-        final Map<String, Object> props = new HashMap<>();
-        KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props);
-        props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
+        final KafkaPropertyProvider propertyProvider = new StandardKafkaPropertyProvider(ConsumerConfig.class);
+        final Map<String, Object> props = propertyProvider.getProperties(context);
+        props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, Boolean.FALSE.toString());
         props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
         props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
 
@@ -352,8 +365,8 @@ public class ConsumeKafka_2_0 extends AbstractProcessor {
         final String topicType = context.getProperty(ConsumeKafka_2_0.TOPIC_TYPE).evaluateAttributeExpressions().getValue();
         final List<String> topics = new ArrayList<>();
         final String keyEncoding = context.getProperty(KEY_ATTRIBUTE_ENCODING).getValue();
-        final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue();
-        final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
+        final String securityProtocol = context.getProperty(SECURITY_PROTOCOL).getValue();
+        final String bootstrapServers = context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
         final boolean honorTransactions = context.getProperty(HONOR_TRANSACTIONS).asBoolean();
         final int commsTimeoutMillis = context.getProperty(COMMS_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue();
         props.put(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG, commsTimeoutMillis);
@@ -388,7 +401,7 @@ public class ConsumeKafka_2_0 extends AbstractProcessor {
             return new ConsumerPool(maxLeases, demarcator, separateByKey, props, topicPattern, maxUncommittedTime, keyEncoding, securityProtocol,
                 bootstrapServers, log, honorTransactions, charset, headerNamePattern, partitionsToConsume);
         } else {
-            getLogger().error("Subscription type has an unknown value {}", new Object[] {topicType});
+            getLogger().error("Subscription type has an unknown value {}", topicType);
             return null;
         }
     }
@@ -413,12 +426,12 @@ public class ConsumeKafka_2_0 extends AbstractProcessor {
         if (!activeLeases.isEmpty()) {
             int count = 0;
             for (final ConsumerLease lease : activeLeases) {
-                getLogger().info("Consumer {} has not finished after waiting 30 seconds; will attempt to wake-up the lease", new Object[] {lease});
+                getLogger().info("Consumer {} has not finished after waiting 30 seconds; will attempt to wake-up the lease", lease);
                 lease.wakeup();
                 count++;
             }
 
-            getLogger().info("Woke up {} consumers", new Object[] {count});
+            getLogger().info("Woke up {} consumers", count);
         }
 
         activeLeases.clear();
@@ -450,7 +463,7 @@ public class ConsumeKafka_2_0 extends AbstractProcessor {
                 }
             } catch (final WakeupException we) {
                 getLogger().warn("Was interrupted while trying to communicate with Kafka with lease {}. "
-                    + "Will roll back session and discard any partially received data.", new Object[] {lease});
+                    + "Will roll back session and discard any partially received data.", lease);
             } catch (final KafkaException kex) {
                 getLogger().error("Exception while interacting with Kafka so will close the lease {} due to {}",
                         new Object[]{lease, kex}, kex);
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
index c87852b48f..e7f6459990 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
@@ -27,6 +27,9 @@ import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.header.Header;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute;
+import org.apache.nifi.kafka.shared.attribute.StandardTransitUriProvider;
+import org.apache.nifi.kafka.shared.property.KeyEncoding;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.exception.ProcessException;
@@ -62,8 +65,6 @@ import java.util.stream.Collectors;
 
 import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_2_0.REL_PARSE_FAILURE;
 import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_2_0.REL_SUCCESS;
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING;
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING;
 
 /**
  * This class represents a lease to access a Kafka Consumer object. The lease is
@@ -147,7 +148,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
      */
     @Override
     public void onPartitionsRevoked(final Collection<TopicPartition> partitions) {
-        logger.debug("Rebalance Alert: Partitions '{}' revoked for lease '{}' with consumer '{}'", new Object[]{partitions, this, kafkaConsumer});
+        logger.debug("Rebalance Alert: Partitions '{}' revoked for lease '{}' with consumer '{}'", partitions, this, kafkaConsumer);
         //force a commit here.  Can reuse the session and consumer after this but must commit now to avoid duplicates if kafka reassigns partition
         commit();
     }
@@ -161,7 +162,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
      */
     @Override
     public void onPartitionsAssigned(final Collection<TopicPartition> partitions) {
-        logger.debug("Rebalance Alert: Partitions '{}' assigned for lease '{}' with consumer '{}'", new Object[]{partitions, this, kafkaConsumer});
+        logger.debug("Rebalance Alert: Partitions '{}' assigned for lease '{}' with consumer '{}'", partitions, this, kafkaConsumer);
     }
 
     public List<TopicPartition> getAssignedPartitions() {
@@ -200,7 +201,6 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
      * higher performance than the other commitOffsets call as it allows the
      * kafka client to collect more data from Kafka before committing the
      * offsets.
-     *
      * if false then we didn't do anything and should probably yield if true
      * then we committed new data
      *
@@ -358,9 +358,9 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
             return null;
         }
 
-        if (HEX_ENCODING.getValue().equals(encoding)) {
+        if (KeyEncoding.HEX.getValue().equals(encoding)) {
             return DatatypeConverter.printHexBinary(key);
-        } else if (UTF8_ENCODING.getValue().equals(encoding)) {
+        } else if (KeyEncoding.UTF8.getValue().equals(encoding)) {
             return new String(key, StandardCharsets.UTF_8);
         } else {
             return null;  // won't happen because it is guaranteed by the Allowable Values
@@ -474,10 +474,10 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
     private void handleParseFailure(final ConsumerRecord<byte[], byte[]> consumerRecord, final ProcessSession session, final Exception cause, final String message) {
         // If we are unable to parse the data, we need to transfer it to 'parse failure' relationship
         final Map<String, String> attributes = getAttributes(consumerRecord);
-        attributes.put(KafkaProcessorUtils.KAFKA_OFFSET, String.valueOf(consumerRecord.offset()));
-        attributes.put(KafkaProcessorUtils.KAFKA_TIMESTAMP, String.valueOf(consumerRecord.timestamp()));
-        attributes.put(KafkaProcessorUtils.KAFKA_PARTITION, String.valueOf(consumerRecord.partition()));
-        attributes.put(KafkaProcessorUtils.KAFKA_TOPIC, consumerRecord.topic());
+        attributes.put(KafkaFlowFileAttribute.KAFKA_OFFSET, String.valueOf(consumerRecord.offset()));
+        attributes.put(KafkaFlowFileAttribute.KAFKA_TIMESTAMP, String.valueOf(consumerRecord.timestamp()));
+        attributes.put(KafkaFlowFileAttribute.KAFKA_PARTITION, String.valueOf(consumerRecord.partition()));
+        attributes.put(KafkaFlowFileAttribute.KAFKA_TOPIC, consumerRecord.topic());
 
         FlowFile failureFlowFile = session.create();
 
@@ -487,7 +487,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
         }
         failureFlowFile = session.putAllAttributes(failureFlowFile, attributes);
 
-        final String transitUri = KafkaProcessorUtils.buildTransitURI(securityProtocol, bootstrapServers, consumerRecord.topic());
+        final String transitUri = StandardTransitUriProvider.getTransitUri(securityProtocol, bootstrapServers, consumerRecord.topic());
         session.getProvenanceReporter().receive(failureFlowFile, transitUri);
 
         session.transfer(failureFlowFile, REL_PARSE_FAILURE);
@@ -594,7 +594,6 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
                         }
                     } catch (final IOException | MalformedRecordException | SchemaValidationException e) {
                         handleParseFailure(consumerRecord, session, e);
-                        continue;
                     }
                 }
             }
@@ -636,32 +635,32 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
 
     private void populateAttributes(final BundleTracker tracker) {
         final Map<String, String> kafkaAttrs = new HashMap<>();
-        kafkaAttrs.put(KafkaProcessorUtils.KAFKA_OFFSET, String.valueOf(tracker.initialOffset));
-        kafkaAttrs.put(KafkaProcessorUtils.KAFKA_TIMESTAMP, String.valueOf(tracker.initialTimestamp));
+        kafkaAttrs.put(KafkaFlowFileAttribute.KAFKA_OFFSET, String.valueOf(tracker.initialOffset));
+        kafkaAttrs.put(KafkaFlowFileAttribute.KAFKA_TIMESTAMP, String.valueOf(tracker.initialTimestamp));
 
         // If we have a kafka key, we will add it as an attribute only if
         // the FlowFile contains a single Record, or if the Records have been separated by Key,
         // because we then know that even though there are multiple Records, they all have the same key.
         if (tracker.key != null && (tracker.totalRecords == 1 || separateByKey)) {
-            if (!keyEncoding.equalsIgnoreCase(KafkaProcessorUtils.DO_NOT_ADD_KEY_AS_ATTRIBUTE.getValue())) {
-                kafkaAttrs.put(KafkaProcessorUtils.KAFKA_KEY, tracker.key);
+            if (!keyEncoding.equalsIgnoreCase(KeyEncoding.DO_NOT_ADD.getValue())) {
+                kafkaAttrs.put(KafkaFlowFileAttribute.KAFKA_KEY, tracker.key);
             }
         }
 
-        kafkaAttrs.put(KafkaProcessorUtils.KAFKA_PARTITION, String.valueOf(tracker.partition));
-        kafkaAttrs.put(KafkaProcessorUtils.KAFKA_TOPIC, tracker.topic);
+        kafkaAttrs.put(KafkaFlowFileAttribute.KAFKA_PARTITION, String.valueOf(tracker.partition));
+        kafkaAttrs.put(KafkaFlowFileAttribute.KAFKA_TOPIC, tracker.topic);
         if (tracker.totalRecords > 1) {
             // Add a record.count attribute to remain consistent with other record-oriented processors. If not
             // reading/writing records, then use "kafka.count" attribute.
             if (tracker.recordWriter == null) {
-                kafkaAttrs.put(KafkaProcessorUtils.KAFKA_COUNT, String.valueOf(tracker.totalRecords));
+                kafkaAttrs.put(KafkaFlowFileAttribute.KAFKA_COUNT, String.valueOf(tracker.totalRecords));
             } else {
                 kafkaAttrs.put("record.count", String.valueOf(tracker.totalRecords));
             }
         }
         final FlowFile newFlowFile = getProcessSession().putAllAttributes(tracker.flowFile, kafkaAttrs);
         final long executionDurationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - leaseStartNanos);
-        final String transitUri = KafkaProcessorUtils.buildTransitURI(securityProtocol, bootstrapServers, tracker.topic);
+        final String transitUri = StandardTransitUriProvider.getTransitUri(securityProtocol, bootstrapServers, tracker.topic);
         getProcessSession().getProvenanceReporter().receive(newFlowFile, transitUri, executionDurationMillis);
         tracker.updateFlowFile(newFlowFile);
     }
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
deleted file mode 100644
index 5c378a5715..0000000000
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
+++ /dev/null
@@ -1,557 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.kafka.pubsub;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.kafka.clients.CommonClientConfigs;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.common.config.SaslConfigs;
-import org.apache.kafka.common.config.SslConfigs;
-import org.apache.kafka.common.serialization.ByteArrayDeserializer;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.apache.nifi.components.AllowableValue;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.components.Validator;
-import org.apache.nifi.components.resource.ResourceCardinality;
-import org.apache.nifi.components.resource.ResourceType;
-import org.apache.nifi.expression.ExpressionLanguageScope;
-import org.apache.nifi.kerberos.KerberosCredentialsService;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.ssl.SSLContextService;
-import org.apache.nifi.util.FormatUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.reflect.Field;
-import java.lang.reflect.Modifier;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Supplier;
-import java.util.regex.Pattern;
-
-public final class KafkaProcessorUtils {
-    private static final String ALLOW_EXPLICIT_KEYTAB = "NIFI_ALLOW_EXPLICIT_KEYTAB";
-
-    final Logger logger = LoggerFactory.getLogger(this.getClass());
-
-    static final AllowableValue UTF8_ENCODING = new AllowableValue("utf-8", "UTF-8 Encoded", "The key is interpreted as a UTF-8 Encoded string.");
-    static final AllowableValue HEX_ENCODING = new AllowableValue("hex", "Hex Encoded",
-            "The key is interpreted as arbitrary binary data and is encoded using hexadecimal characters with uppercase letters");
-    static final AllowableValue DO_NOT_ADD_KEY_AS_ATTRIBUTE = new AllowableValue("do-not-add", "Do Not Add Key as Attribute",
-        "The key will not be added as an Attribute");
-
-    static final Pattern HEX_KEY_PATTERN = Pattern.compile("(?:[0123456789abcdefABCDEF]{2})+");
-
-    static final String KAFKA_KEY = "kafka.key";
-    static final String KAFKA_TOPIC = "kafka.topic";
-    static final String KAFKA_PARTITION = "kafka.partition";
-    static final String KAFKA_OFFSET = "kafka.offset";
-    static final String KAFKA_TIMESTAMP = "kafka.timestamp";
-    static final String KAFKA_COUNT = "kafka.count";
-
-    static final AllowableValue SEC_PLAINTEXT = new AllowableValue("PLAINTEXT", "PLAINTEXT", "PLAINTEXT");
-    static final AllowableValue SEC_SSL = new AllowableValue("SSL", "SSL", "SSL");
-    public static final AllowableValue SEC_SASL_PLAINTEXT = new AllowableValue("SASL_PLAINTEXT", "SASL_PLAINTEXT", "SASL_PLAINTEXT");
-    public static final AllowableValue SEC_SASL_SSL = new AllowableValue("SASL_SSL", "SASL_SSL", "SASL_SSL");
-
-    static final String GSSAPI_VALUE = "GSSAPI";
-    static final AllowableValue SASL_MECHANISM_GSSAPI = new AllowableValue(GSSAPI_VALUE, GSSAPI_VALUE,
-            "The mechanism for authentication via Kerberos. The principal and keytab must be provided to the processor " +
-                    "by using a Keytab Credential service, or by specifying the properties directly in the processor.");
-
-    static final String PLAIN_VALUE = "PLAIN";
-    static final AllowableValue SASL_MECHANISM_PLAIN = new AllowableValue(PLAIN_VALUE, PLAIN_VALUE,
-            "The mechanism for authentication via username and password. The username and password properties must " +
-                    "be populated when using this mechanism.");
-
-    static final String SCRAM_SHA256_VALUE = "SCRAM-SHA-256";
-    static final AllowableValue SASL_MECHANISM_SCRAM_SHA256 = new AllowableValue(SCRAM_SHA256_VALUE, SCRAM_SHA256_VALUE,"The Salted Challenge Response Authentication Mechanism using SHA-256. " +
-            "The username and password properties must be set when using this mechanism.");
-
-    static final String SCRAM_SHA512_VALUE = "SCRAM-SHA-512";
-    static final AllowableValue SASL_MECHANISM_SCRAM_SHA512 = new AllowableValue(SCRAM_SHA512_VALUE, SCRAM_SHA512_VALUE,"The Salted Challenge Response Authentication Mechanism using SHA-512. " +
-            "The username and password properties must be set when using this mechanism.");
-
-    static final AllowableValue FAILURE_STRATEGY_FAILURE_RELATIONSHIP = new AllowableValue("Route to Failure", "Route to Failure",
-        "When unable to publish a FlowFile to Kafka, the FlowFile will be routed to the 'failure' relationship.");
-    static final AllowableValue FAILURE_STRATEGY_ROLLBACK = new AllowableValue("Rollback", "Rollback",
-        "When unable to publish a FlowFile to Kafka, the FlowFile will be placed back on the top of its queue so that it will be the next FlowFile tried again. " +
-            "For dataflows where ordering of FlowFiles is important, this strategy can be used along with ensuring that the each processor in the dataflow uses only a single Concurrent Task.");
-
-    public static final PropertyDescriptor BOOTSTRAP_SERVERS = new PropertyDescriptor.Builder()
-            .name(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)
-            .displayName("Kafka Brokers")
-            .description("A comma-separated list of known Kafka Brokers in the format <host>:<port>")
-            .required(true)
-            .addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-            .defaultValue("localhost:9092")
-            .build();
-    public static final PropertyDescriptor SECURITY_PROTOCOL = new PropertyDescriptor.Builder()
-            .name("security.protocol")
-            .displayName("Security Protocol")
-            .description("Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.")
-            .required(true)
-            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .allowableValues(SEC_PLAINTEXT, SEC_SSL, SEC_SASL_PLAINTEXT, SEC_SASL_SSL)
-            .defaultValue(SEC_PLAINTEXT.getValue())
-            .build();
-    static final PropertyDescriptor SASL_MECHANISM = new PropertyDescriptor.Builder()
-            .name("sasl.mechanism")
-            .displayName("SASL Mechanism")
-            .description("The SASL mechanism to use for authentication. Corresponds to Kafka's 'sasl.mechanism' property.")
-            .required(true)
-            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .allowableValues(SASL_MECHANISM_GSSAPI, SASL_MECHANISM_PLAIN, SASL_MECHANISM_SCRAM_SHA256, SASL_MECHANISM_SCRAM_SHA512)
-            .defaultValue(GSSAPI_VALUE)
-            .build();
-    public static final PropertyDescriptor JAAS_SERVICE_NAME = new PropertyDescriptor.Builder()
-            .name("sasl.kerberos.service.name")
-            .displayName("Kerberos Service Name")
-            .description("The service name that matches the primary name of the Kafka server configured in the broker JAAS file."
-                    + "This can be defined either in Kafka's JAAS config or in Kafka's config. "
-                    + "Corresponds to Kafka's 'security.protocol' property."
-                    + "It is ignored unless one of the SASL options of the <Security Protocol> are selected.")
-            .required(false)
-            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-            .build();
-    static final PropertyDescriptor USER_PRINCIPAL = new PropertyDescriptor.Builder()
-            .name("sasl.kerberos.principal")
-            .displayName("Kerberos Principal")
-            .description("The Kerberos principal that will be used to connect to brokers. If not set, it is expected to set a JAAS configuration file "
-                    + "in the JVM properties defined in the bootstrap.conf file. This principal will be set into 'sasl.jaas.config' Kafka's property.")
-            .required(false)
-            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-            .build();
-    static final PropertyDescriptor USER_KEYTAB = new PropertyDescriptor.Builder()
-            .name("sasl.kerberos.keytab")
-            .displayName("Kerberos Keytab")
-            .description("The Kerberos keytab that will be used to connect to brokers. If not set, it is expected to set a JAAS configuration file "
-                    + "in the JVM properties defined in the bootstrap.conf file. This principal will be set into 'sasl.jaas.config' Kafka's property.")
-            .required(false)
-            .identifiesExternalResource(ResourceCardinality.SINGLE, ResourceType.FILE)
-            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-            .build();
-    static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
-            .name("sasl.username")
-            .displayName("Username")
-            .description("The username when the SASL Mechanism is " + PLAIN_VALUE + " or " + SCRAM_SHA256_VALUE + "/" + SCRAM_SHA512_VALUE)
-            .required(false)
-            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-            .build();
-    static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
-            .name("sasl.password")
-            .displayName("Password")
-            .description("The password for the given username when the SASL Mechanism is " + PLAIN_VALUE + " or " + SCRAM_SHA256_VALUE + "/" + SCRAM_SHA512_VALUE)
-            .required(false)
-            .sensitive(true)
-            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-            .build();
-    static final PropertyDescriptor TOKEN_AUTH = new PropertyDescriptor.Builder()
-            .name("sasl.token.auth")
-            .displayName("Token Auth")
-            .description("When " + SASL_MECHANISM.getDisplayName() + " is " + SCRAM_SHA256_VALUE + " or " + SCRAM_SHA512_VALUE
-                    + ", this property indicates if token authentication should be used.")
-            .required(false)
-            .allowableValues("true", "false")
-            .defaultValue("false")
-            .build();
-    public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
-            .name("ssl.context.service")
-            .displayName("SSL Context Service")
-            .description("Specifies the SSL Context Service to use for communicating with Kafka.")
-            .required(false)
-            .identifiesControllerService(SSLContextService.class)
-            .build();
-    public static final PropertyDescriptor KERBEROS_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
-        .name("kerberos-credentials-service")
-        .displayName("Kerberos Credentials Service")
-        .description("Specifies the Kerberos Credentials Controller Service that should be used for authenticating with Kerberos")
-        .identifiesControllerService(KerberosCredentialsService.class)
-        .required(false)
-        .build();
-
-    static final PropertyDescriptor FAILURE_STRATEGY = new PropertyDescriptor.Builder()
-        .name("Failure Strategy")
-        .displayName("Failure Strategy")
-        .description("Dictates how the processor handles a FlowFile if it is unable to publish the data to Kafka")
-        .required(true)
-        .allowableValues(FAILURE_STRATEGY_FAILURE_RELATIONSHIP, FAILURE_STRATEGY_ROLLBACK)
-        .defaultValue(FAILURE_STRATEGY_FAILURE_RELATIONSHIP.getValue())
-        .build();
-
-    static List<PropertyDescriptor> getCommonPropertyDescriptors() {
-        return Arrays.asList(
-                BOOTSTRAP_SERVERS,
-                SECURITY_PROTOCOL,
-                SASL_MECHANISM,
-                JAAS_SERVICE_NAME,
-                KERBEROS_CREDENTIALS_SERVICE,
-                USER_PRINCIPAL,
-                USER_KEYTAB,
-                USERNAME,
-                PASSWORD,
-                TOKEN_AUTH,
-                SSL_CONTEXT_SERVICE
-        );
-    }
-
-    public static Collection<ValidationResult> validateCommonProperties(final ValidationContext validationContext) {
-        List<ValidationResult> results = new ArrayList<>();
-
-        final String securityProtocol = validationContext.getProperty(SECURITY_PROTOCOL).getValue();
-        final String saslMechanism = validationContext.getProperty(SASL_MECHANISM).getValue();
-
-        final String explicitPrincipal = validationContext.getProperty(USER_PRINCIPAL).evaluateAttributeExpressions().getValue();
-        final String explicitKeytab = validationContext.getProperty(USER_KEYTAB).evaluateAttributeExpressions().getValue();
-        final KerberosCredentialsService credentialsService = validationContext.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
-
-        final String resolvedPrincipal;
-        final String resolvedKeytab;
-        if (credentialsService == null) {
-            resolvedPrincipal = explicitPrincipal;
-            resolvedKeytab = explicitKeytab;
-        } else {
-            resolvedPrincipal = credentialsService.getPrincipal();
-            resolvedKeytab = credentialsService.getKeytab();
-        }
-
-        if (credentialsService != null && (explicitPrincipal != null || explicitKeytab != null)) {
-            results.add(new ValidationResult.Builder()
-                .subject("Kerberos Credentials")
-                .valid(false)
-                .explanation("Cannot specify both a Kerberos Credentials Service and a principal/keytab")
-                .build());
-        }
-
-        final String allowExplicitKeytabVariable = System.getenv(ALLOW_EXPLICIT_KEYTAB);
-        if ("false".equalsIgnoreCase(allowExplicitKeytabVariable) && (explicitPrincipal != null || explicitKeytab != null)) {
-            results.add(new ValidationResult.Builder()
-                .subject("Kerberos Credentials")
-                .valid(false)
-                .explanation("The '" + ALLOW_EXPLICIT_KEYTAB + "' system environment variable is configured to forbid explicitly configuring principal/keytab in processors. "
-                    + "The Kerberos Credentials Service should be used instead of setting the Kerberos Keytab or Kerberos Principal property.")
-                .build());
-        }
-
-        // validates that if the SASL mechanism is GSSAPI (kerberos) AND one of the SASL options is selected
-        // for security protocol, then Kerberos principal is provided as well
-        if (SASL_MECHANISM_GSSAPI.getValue().equals(saslMechanism)
-                && (SEC_SASL_PLAINTEXT.getValue().equals(securityProtocol) || SEC_SASL_SSL.getValue().equals(securityProtocol))) {
-            String jaasServiceName = validationContext.getProperty(JAAS_SERVICE_NAME).evaluateAttributeExpressions().getValue();
-            if (jaasServiceName == null || jaasServiceName.trim().length() == 0) {
-                results.add(new ValidationResult.Builder().subject(JAAS_SERVICE_NAME.getDisplayName()).valid(false)
-                    .explanation("The <" + JAAS_SERVICE_NAME.getDisplayName() + "> property must be set when <"
-                        + SECURITY_PROTOCOL.getDisplayName() + "> is configured as '"
-                        + SEC_SASL_PLAINTEXT.getValue() + "' or '" + SEC_SASL_SSL.getValue() + "'.")
-                    .build());
-            }
-
-            if ((resolvedKeytab == null && resolvedPrincipal != null) || (resolvedKeytab != null && resolvedPrincipal == null)) {
-                results.add(new ValidationResult.Builder()
-                    .subject(JAAS_SERVICE_NAME.getDisplayName())
-                    .valid(false)
-                    .explanation("Both <" + USER_KEYTAB.getDisplayName() + "> and <" + USER_PRINCIPAL.getDisplayName() + "> "
-                        + "must be set or neither must be set.")
-                    .build());
-            }
-        }
-
-        // validate that if SASL Mechanism is PLAIN or SCRAM, then username and password are both provided
-        if (SASL_MECHANISM_PLAIN.getValue().equals(saslMechanism)
-                || SASL_MECHANISM_SCRAM_SHA256.getValue().equals(saslMechanism)
-                || SASL_MECHANISM_SCRAM_SHA512.getValue().equals(saslMechanism)) {
-            final String username = validationContext.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
-            if (StringUtils.isBlank(username)) {
-                results.add(new ValidationResult.Builder()
-                        .subject(USERNAME.getDisplayName())
-                        .valid(false)
-                        .explanation("A username is required when " + SASL_MECHANISM.getDisplayName()
-                                + " is " + PLAIN_VALUE + " or " + SCRAM_SHA256_VALUE + "/" + SCRAM_SHA512_VALUE)
-                        .build());
-            }
-
-            final String password = validationContext.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
-            if (StringUtils.isBlank(password)) {
-                results.add(new ValidationResult.Builder()
-                        .subject(PASSWORD.getDisplayName())
-                        .valid(false)
-                        .explanation("A password is required when " + SASL_MECHANISM.getDisplayName()
-                                + " is " + PLAIN_VALUE + " or " + SCRAM_SHA256_VALUE + "/" + SCRAM_SHA512_VALUE)
-                        .build());
-            }
-        }
-
-        // If SSL or SASL_SSL then SSLContext Controller Service must be set.
-        final boolean sslProtocol = SEC_SSL.getValue().equals(securityProtocol) || SEC_SASL_SSL.getValue().equals(securityProtocol);
-        final boolean csSet = validationContext.getProperty(SSL_CONTEXT_SERVICE).isSet();
-        if (csSet && !sslProtocol) {
-            results.add(new ValidationResult.Builder()
-                .subject(SECURITY_PROTOCOL.getDisplayName())
-                .valid(false)
-                .explanation("If you set the SSL Controller Service you should also choose an SSL based security protocol.")
-                .build());
-        }
-
-        if (!csSet && sslProtocol) {
-            results.add(new ValidationResult.Builder()
-                .subject(SSL_CONTEXT_SERVICE.getDisplayName())
-                .valid(false)
-                .explanation("If you set to an SSL based protocol you need to set the SSL Controller Service")
-                .build());
-        }
-
-        final String enableAutoCommit = validationContext.getProperty(new PropertyDescriptor.Builder().name(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG).build()).getValue();
-        if (enableAutoCommit != null && !enableAutoCommit.toLowerCase().equals("false")) {
-            results.add(new ValidationResult.Builder().subject(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)
-                .explanation("Enable auto commit must be false. It is managed by the processor.").build());
-        }
-
-        final String keySerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG).build()).getValue();
-        if (keySerializer != null && !ByteArraySerializer.class.getName().equals(keySerializer)) {
-            results.add(new ValidationResult.Builder().subject(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)
-                .explanation("Key Serializer must be " + ByteArraySerializer.class.getName() + "' was '" + keySerializer + "'").build());
-        }
-
-        final String valueSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG).build()).getValue();
-        if (valueSerializer != null && !ByteArraySerializer.class.getName().equals(valueSerializer)) {
-            results.add(new ValidationResult.Builder().subject(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)
-                .explanation("Value Serializer must be " + ByteArraySerializer.class.getName() + "' was '" + valueSerializer + "'").build());
-        }
-
-        final String keyDeSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG).build()).getValue();
-        if (keyDeSerializer != null && !ByteArrayDeserializer.class.getName().equals(keyDeSerializer)) {
-            results.add(new ValidationResult.Builder().subject(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)
-                .explanation("Key De-Serializer must be '" + ByteArrayDeserializer.class.getName() + "' was '" + keyDeSerializer + "'").build());
-        }
-
-        final String valueDeSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG).build()).getValue();
-        if (valueDeSerializer != null && !ByteArrayDeserializer.class.getName().equals(valueDeSerializer)) {
-            results.add(new ValidationResult.Builder().subject(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG)
-                .explanation("Value De-Serializer must be " + ByteArrayDeserializer.class.getName() + "' was '" + valueDeSerializer + "'").build());
-        }
-
-        return results;
-    }
-
-    public static final class KafkaConfigValidator implements Validator {
-
-        final Class<?> classType;
-
-        public KafkaConfigValidator(final Class<?> classType) {
-            this.classType = classType;
-        }
-
-        @Override
-        public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
-            if (subject.startsWith(ConsumerPartitionsUtil.PARTITION_PROPERTY_NAME_PREFIX)) {
-                return new ValidationResult.Builder().valid(true).build();
-            }
-
-            final boolean knownValue = KafkaProcessorUtils.isStaticStringFieldNamePresent(subject, classType, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class);
-            return new ValidationResult.Builder().subject(subject).explanation("Must be a known configuration parameter for this kafka client").valid(knownValue).build();
-        }
-    }
-
-    /**
-     * Builds transit URI for provenance event. The transit URI will be in the
-     * form of &lt;security.protocol&gt;://&lt;bootstrap.servers&gt;/topic
-     */
-    static String buildTransitURI(String securityProtocol, String brokers, String topic) {
-        StringBuilder builder = new StringBuilder();
-        builder.append(securityProtocol);
-        builder.append("://");
-        builder.append(brokers);
-        builder.append("/");
-        builder.append(topic);
-        return builder.toString();
-    }
-
-
-    static void buildCommonKafkaProperties(final ProcessContext context, final Class<?> kafkaConfigClass, final Map<String, Object> mapToPopulate) {
-        for (PropertyDescriptor propertyDescriptor : context.getProperties().keySet()) {
-            if (propertyDescriptor.equals(SSL_CONTEXT_SERVICE)) {
-                // Translate SSLContext Service configuration into Kafka properties
-                final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
-                if (sslContextService != null && sslContextService.isKeyStoreConfigured()) {
-                    mapToPopulate.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, sslContextService.getKeyStoreFile());
-                    mapToPopulate.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, sslContextService.getKeyStorePassword());
-                    final String keyPass = sslContextService.getKeyPassword() == null ? sslContextService.getKeyStorePassword() : sslContextService.getKeyPassword();
-                    mapToPopulate.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, keyPass);
-                    mapToPopulate.put(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, sslContextService.getKeyStoreType());
-                }
-
-                if (sslContextService != null && sslContextService.isTrustStoreConfigured()) {
-                    mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, sslContextService.getTrustStoreFile());
-                    mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, sslContextService.getTrustStorePassword());
-                    mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, sslContextService.getTrustStoreType());
-                }
-            }
-
-            String propertyName = propertyDescriptor.getName();
-            String propertyValue = propertyDescriptor.isExpressionLanguageSupported()
-                    ? context.getProperty(propertyDescriptor).evaluateAttributeExpressions().getValue()
-                    : context.getProperty(propertyDescriptor).getValue();
-
-            if (propertyValue != null && !propertyName.equals(USER_PRINCIPAL.getName()) && !propertyName.equals(USER_KEYTAB.getName())
-                && !propertyName.startsWith(ConsumerPartitionsUtil.PARTITION_PROPERTY_NAME_PREFIX)) {
-
-                // If the property name ends in ".ms" then it is a time period. We want to accept either an integer as number of milliseconds
-                // or the standard NiFi time period such as "5 secs"
-                if (propertyName.endsWith(".ms") && !StringUtils.isNumeric(propertyValue.trim())) { // kafka standard time notation
-                    propertyValue = String.valueOf(FormatUtils.getTimeDuration(propertyValue.trim(), TimeUnit.MILLISECONDS));
-                }
-
-                if (isStaticStringFieldNamePresent(propertyName, kafkaConfigClass, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class)) {
-                    mapToPopulate.put(propertyName, propertyValue);
-                }
-            }
-        }
-
-        String securityProtocol = context.getProperty(SECURITY_PROTOCOL).getValue();
-        if (SEC_SASL_PLAINTEXT.getValue().equals(securityProtocol) || SEC_SASL_SSL.getValue().equals(securityProtocol)) {
-            setJaasConfig(mapToPopulate, context);
-        }
-    }
-
-    /**
-     * Method used to create a transactional id Supplier for KafkaProducer
-     *
-     * @param prefix String transactional id prefix, can be null
-     * @return A Supplier that generates transactional id
-     */
-    static Supplier<String> getTransactionalIdSupplier(String prefix) {
-        return () -> (prefix == null ? "" : prefix)  + UUID.randomUUID().toString();
-    }
-
-    /**
-     * Method used to configure the 'sasl.jaas.config' property based on KAFKA-4259<br />
-     * https://cwiki.apache.org/confluence/display/KAFKA/KIP-85%3A+Dynamic+JAAS+configuration+for+Kafka+clients<br />
-     * <br />
-     * It expects something with the following format: <br />
-     * <br />
-     * &lt;LoginModuleClass&gt; &lt;ControlFlag&gt; *(&lt;OptionName&gt;=&lt;OptionValue&gt;); <br />
-     * ControlFlag = required / requisite / sufficient / optional
-     *
-     * @param mapToPopulate Map of configuration properties
-     * @param context Context
-     */
-    private static void setJaasConfig(Map<String, Object> mapToPopulate, ProcessContext context) {
-        final String saslMechanism = context.getProperty(SASL_MECHANISM).getValue();
-        switch (saslMechanism) {
-            case GSSAPI_VALUE:
-                setGssApiJaasConfig(mapToPopulate, context);
-                break;
-            case PLAIN_VALUE:
-                setPlainJaasConfig(mapToPopulate, context);
-                break;
-            case SCRAM_SHA256_VALUE:
-            case SCRAM_SHA512_VALUE:
-                setScramJaasConfig(mapToPopulate, context);
-                break;
-            default:
-                throw new IllegalStateException("Unknown " + SASL_MECHANISM.getDisplayName() + ": " + saslMechanism);
-        }
-    }
-
-    private static void setGssApiJaasConfig(final Map<String, Object> mapToPopulate, final ProcessContext context) {
-        String keytab = context.getProperty(USER_KEYTAB).evaluateAttributeExpressions().getValue();
-        String principal = context.getProperty(USER_PRINCIPAL).evaluateAttributeExpressions().getValue();
-
-        // If the Kerberos Credentials Service is specified, we need to use its configuration, not the explicit properties for principal/keytab.
-        // The customValidate method ensures that only one can be set, so we know that the principal & keytab above are null.
-        final KerberosCredentialsService credentialsService = context.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
-        if (credentialsService != null) {
-            principal = credentialsService.getPrincipal();
-            keytab = credentialsService.getKeytab();
-        }
-
-
-        String serviceName = context.getProperty(JAAS_SERVICE_NAME).evaluateAttributeExpressions().getValue();
-        if (StringUtils.isNotBlank(keytab) && StringUtils.isNotBlank(principal) && StringUtils.isNotBlank(serviceName)) {
-            mapToPopulate.put(SaslConfigs.SASL_JAAS_CONFIG, "com.sun.security.auth.module.Krb5LoginModule required "
-                    + "useTicketCache=false "
-                    + "renewTicket=true "
-                    + "serviceName=\"" + serviceName + "\" "
-                    + "useKeyTab=true "
-                    + "keyTab=\"" + keytab + "\" "
-                    + "principal=\"" + principal + "\";");
-        }
-    }
-
-    private static void setPlainJaasConfig(final Map<String, Object> mapToPopulate, final ProcessContext context) {
-        final String username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
-        final String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
-
-        mapToPopulate.put(SaslConfigs.SASL_JAAS_CONFIG, "org.apache.kafka.common.security.plain.PlainLoginModule required "
-                + "username=\"" + username + "\" "
-                + "password=\"" + password + "\";");
-    }
-
-    private static void setScramJaasConfig(final Map<String, Object> mapToPopulate, final ProcessContext context) {
-        final String username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
-        final String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
-
-        final StringBuilder builder = new StringBuilder("org.apache.kafka.common.security.scram.ScramLoginModule required ")
-                .append("username=\"" + username + "\" ")
-                .append("password=\"" + password + "\"");
-
-        final Boolean tokenAuth = context.getProperty(TOKEN_AUTH).asBoolean();
-        if (tokenAuth != null && tokenAuth) {
-            builder.append(" tokenauth=\"true\"");
-        }
-
-        builder.append(";");
-        mapToPopulate.put(SaslConfigs.SASL_JAAS_CONFIG, builder.toString());
-    }
-
-    public static boolean isStaticStringFieldNamePresent(final String name, final Class<?>... classes) {
-        return KafkaProcessorUtils.getPublicStaticStringFieldValues(classes).contains(name);
-    }
-
-    private static Set<String> getPublicStaticStringFieldValues(final Class<?>... classes) {
-        final Set<String> strings = new HashSet<>();
-        for (final Class<?> classType : classes) {
-            for (final Field field : classType.getDeclaredFields()) {
-                if (Modifier.isPublic(field.getModifiers()) && Modifier.isStatic(field.getModifiers()) && field.getType().equals(String.class)) {
-                    try {
-                        strings.add(String.valueOf(field.get(null)));
-                    } catch (IllegalArgumentException | IllegalAccessException ex) {
-                        //ignore
-                    }
-                }
-            }
-        }
-        return strings;
-    }
-
-}
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_0.java
index f659bb456c..b4f398464c 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_0.java
@@ -35,13 +35,20 @@ import org.apache.nifi.components.PropertyDescriptor.Builder;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.kafka.shared.attribute.StandardTransitUriProvider;
+import org.apache.nifi.kafka.shared.component.KafkaPublishComponent;
+import org.apache.nifi.kafka.shared.property.FailureStrategy;
+import org.apache.nifi.kafka.shared.property.provider.KafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.property.provider.StandardKafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.transaction.TransactionIdSupplier;
+import org.apache.nifi.kafka.shared.validation.DynamicPropertyValidator;
+import org.apache.nifi.kafka.shared.validation.KafkaClientCustomValidationFunction;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.util.FlowFileFilters;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.record.path.RecordPath;
@@ -57,13 +64,10 @@ import org.apache.nifi.serialization.record.Record;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.RecordSet;
 
-import java.io.IOException;
-import java.io.InputStream;
 import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -79,8 +83,6 @@ import java.util.regex.Pattern;
 import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
 import static org.apache.nifi.expression.ExpressionLanguageScope.NONE;
 import static org.apache.nifi.expression.ExpressionLanguageScope.VARIABLE_REGISTRY;
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.FAILURE_STRATEGY;
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.FAILURE_STRATEGY_ROLLBACK;
 
 @Tags({"Apache", "Kafka", "Record", "csv", "json", "avro", "logs", "Put", "Send", "Message", "PubSub", "2.0"})
 @CapabilityDescription("Sends the contents of a FlowFile as individual records to Apache Kafka using the Kafka 2.0 Producer API. "
@@ -95,7 +97,7 @@ import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.FAILUR
 @WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Kafka for this FlowFile. This attribute is added only to "
     + "FlowFiles that are routed to success.")
 @SeeAlso({PublishKafka_2_0.class, ConsumeKafka_2_0.class, ConsumeKafkaRecord_2_0.class})
-public class PublishKafkaRecord_2_0 extends AbstractProcessor {
+public class PublishKafkaRecord_2_0 extends AbstractProcessor implements KafkaPublishComponent {
     protected static final String MSG_COUNT = "msg.count";
 
     static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("all", "Guarantee Replicated Delivery",
@@ -286,26 +288,26 @@ public class PublishKafkaRecord_2_0 extends AbstractProcessor {
 
     static {
         final List<PropertyDescriptor> properties = new ArrayList<>();
-        properties.add(KafkaProcessorUtils.BOOTSTRAP_SERVERS);
+        properties.add(BOOTSTRAP_SERVERS);
         properties.add(TOPIC);
         properties.add(RECORD_READER);
         properties.add(RECORD_WRITER);
         properties.add(USE_TRANSACTIONS);
-        properties.add(KafkaProcessorUtils.FAILURE_STRATEGY);
+        properties.add(FAILURE_STRATEGY);
         properties.add(TRANSACTIONAL_ID_PREFIX);
         properties.add(DELIVERY_GUARANTEE);
         properties.add(ATTRIBUTE_NAME_REGEX);
         properties.add(MESSAGE_HEADER_ENCODING);
-        properties.add(KafkaProcessorUtils.SECURITY_PROTOCOL);
-        properties.add(KafkaProcessorUtils.SASL_MECHANISM);
-        properties.add(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE);
-        properties.add(KafkaProcessorUtils.JAAS_SERVICE_NAME);
-        properties.add(KafkaProcessorUtils.USER_PRINCIPAL);
-        properties.add(KafkaProcessorUtils.USER_KEYTAB);
-        properties.add(KafkaProcessorUtils.USERNAME);
-        properties.add(KafkaProcessorUtils.PASSWORD);
-        properties.add(KafkaProcessorUtils.TOKEN_AUTH);
-        properties.add(KafkaProcessorUtils.SSL_CONTEXT_SERVICE);
+        properties.add(SECURITY_PROTOCOL);
+        properties.add(SASL_MECHANISM);
+        properties.add(KERBEROS_CREDENTIALS_SERVICE);
+        properties.add(KERBEROS_SERVICE_NAME);
+        properties.add(KERBEROS_PRINCIPAL);
+        properties.add(KERBEROS_KEYTAB);
+        properties.add(SASL_USERNAME);
+        properties.add(SASL_PASSWORD);
+        properties.add(TOKEN_AUTHENTICATION);
+        properties.add(SSL_CONTEXT_SERVICE);
         properties.add(MESSAGE_KEY_FIELD);
         properties.add(MAX_REQUEST_SIZE);
         properties.add(ACK_WAIT_TIME);
@@ -337,7 +339,7 @@ public class PublishKafkaRecord_2_0 extends AbstractProcessor {
         return new Builder()
             .description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.")
             .name(propertyDescriptorName)
-            .addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class))
+            .addValidator(new DynamicPropertyValidator(ProducerConfig.class))
             .dynamic(true)
             .expressionLanguageSupported(VARIABLE_REGISTRY)
             .build();
@@ -345,8 +347,7 @@ public class PublishKafkaRecord_2_0 extends AbstractProcessor {
 
     @Override
     protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
-        final List<ValidationResult> results = new ArrayList<>();
-        results.addAll(KafkaProcessorUtils.validateCommonProperties(validationContext));
+        final List<ValidationResult> results = new ArrayList<>(new KafkaClientCustomValidationFunction().apply(validationContext));
 
         final boolean useTransactions = validationContext.getProperty(USE_TRANSACTIONS).asBoolean();
         if (useTransactions) {
@@ -401,19 +402,19 @@ public class PublishKafkaRecord_2_0 extends AbstractProcessor {
 
     protected PublisherPool createPublisherPool(final ProcessContext context) {
         final int maxMessageSize = context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue();
-        final long maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS).longValue();
+        final long maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
 
         final String attributeNameRegex = context.getProperty(ATTRIBUTE_NAME_REGEX).getValue();
         final Pattern attributeNamePattern = attributeNameRegex == null ? null : Pattern.compile(attributeNameRegex);
         final boolean useTransactions = context.getProperty(USE_TRANSACTIONS).asBoolean();
         final String transactionalIdPrefix = context.getProperty(TRANSACTIONAL_ID_PREFIX).evaluateAttributeExpressions().getValue();
-        Supplier<String> transactionalIdSupplier = KafkaProcessorUtils.getTransactionalIdSupplier(transactionalIdPrefix);
+        Supplier<String> transactionalIdSupplier = new TransactionIdSupplier(transactionalIdPrefix);
 
         final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
         final Charset charset = Charset.forName(charsetName);
 
-        final Map<String, Object> kafkaProperties = new HashMap<>();
-        KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
+        final KafkaPropertyProvider propertyProvider = new StandardKafkaPropertyProvider(ProducerConfig.class);
+        final Map<String, Object> kafkaProperties = propertyProvider.getProperties(context);
         kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize));
@@ -443,8 +444,8 @@ public class PublishKafkaRecord_2_0 extends AbstractProcessor {
             return;
         }
 
-        final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue();
-        final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
+        final String securityProtocol = context.getProperty(SECURITY_PROTOCOL).getValue();
+        final String bootstrapServers = context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
         final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
         final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
         final boolean useTransactions = context.getProperty(USE_TRANSACTIONS).asBoolean();
@@ -481,24 +482,20 @@ public class PublishKafkaRecord_2_0 extends AbstractProcessor {
                     final Function<Record, Integer> partitioner = getPartitioner(context, flowFile);
 
                     try {
-                        session.read(flowFile, new InputStreamCallback() {
-                            @Override
-                            public void process(final InputStream in) throws IOException {
-                                try {
-                                    final RecordReader reader = readerFactory.createRecordReader(flowFile, in, getLogger());
-                                    final RecordSet recordSet = reader.createRecordSet();
-
-                                    final RecordSchema schema = writerFactory.getSchema(flowFile.getAttributes(), recordSet.getSchema());
-                                    lease.publish(flowFile, recordSet, writerFactory, schema, messageKeyField, topic, partitioner);
-                                } catch (final SchemaNotFoundException | MalformedRecordException e) {
-                                    throw new ProcessException(e);
-                                }
+                        session.read(flowFile, in -> {
+                            try {
+                                final RecordReader reader = readerFactory.createRecordReader(flowFile, in, getLogger());
+                                final RecordSet recordSet = reader.createRecordSet();
+
+                                final RecordSchema schema = writerFactory.getSchema(flowFile.getAttributes(), recordSet.getSchema());
+                                lease.publish(flowFile, recordSet, writerFactory, schema, messageKeyField, topic, partitioner);
+                            } catch (final SchemaNotFoundException | MalformedRecordException e) {
+                                throw new ProcessException(e);
                             }
                         });
                     } catch (final Exception e) {
                         // The FlowFile will be obtained and the error logged below, when calling publishResult.getFailedFlowFiles()
                         lease.fail(flowFile, e);
-                        continue;
                     }
                 }
 
@@ -520,7 +517,7 @@ public class PublishKafkaRecord_2_0 extends AbstractProcessor {
                     success = session.putAttribute(success, MSG_COUNT, String.valueOf(msgCount));
                     session.adjustCounter("Messages Sent", msgCount, true);
 
-                    final String transitUri = KafkaProcessorUtils.buildTransitURI(securityProtocol, bootstrapServers, topic);
+                    final String transitUri = StandardTransitUriProvider.getTransitUri(securityProtocol, bootstrapServers, topic);
                     session.getProvenanceReporter().send(success, transitUri, "Sent " + msgCount + " messages", transmissionMillis);
                     session.transfer(success, REL_SUCCESS);
                 }
@@ -564,7 +561,7 @@ public class PublishKafkaRecord_2_0 extends AbstractProcessor {
 
     private PublishFailureStrategy getFailureStrategy(final ProcessContext context) {
         final String strategy = context.getProperty(FAILURE_STRATEGY).getValue();
-        if (FAILURE_STRATEGY_ROLLBACK.getValue().equals(strategy)) {
+        if (FailureStrategy.ROLLBACK.getValue().equals(strategy)) {
             return (session, flowFiles) -> session.rollback();
         } else {
             return (session, flowFiles) -> session.transfer(flowFiles, REL_FAILURE);
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_2_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_2_0.java
index f3260c1f26..9bc312fc9e 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_2_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_2_0.java
@@ -34,26 +34,33 @@ import org.apache.nifi.components.PropertyDescriptor.Builder;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute;
+import org.apache.nifi.kafka.shared.attribute.StandardTransitUriProvider;
+import org.apache.nifi.kafka.shared.component.KafkaPublishComponent;
+import org.apache.nifi.kafka.shared.property.FailureStrategy;
+import org.apache.nifi.kafka.shared.property.KeyEncoding;
+import org.apache.nifi.kafka.shared.property.provider.KafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.property.provider.StandardKafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.transaction.TransactionIdSupplier;
+import org.apache.nifi.kafka.shared.validation.DynamicPropertyValidator;
+import org.apache.nifi.kafka.shared.validation.KafkaClientCustomValidationFunction;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.util.FlowFileFilters;
 import org.apache.nifi.processor.util.StandardValidators;
 
 import javax.xml.bind.DatatypeConverter;
 import java.io.BufferedInputStream;
-import java.io.IOException;
 import java.io.InputStream;
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -66,8 +73,6 @@ import java.util.regex.Pattern;
 import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
 import static org.apache.nifi.expression.ExpressionLanguageScope.NONE;
 import static org.apache.nifi.expression.ExpressionLanguageScope.VARIABLE_REGISTRY;
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.FAILURE_STRATEGY;
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.FAILURE_STRATEGY_ROLLBACK;
 
 @Tags({"Apache", "Kafka", "Put", "Send", "Message", "PubSub", "2.0"})
 @CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka using the Kafka 2.0 Producer API."
@@ -83,7 +88,7 @@ import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.FAILUR
 @WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Kafka for this FlowFile. This attribute is added only to "
     + "FlowFiles that are routed to success. If the <Message Demarcator> Property is not set, this will always be 1, but if the Property is set, it may "
     + "be greater than 1.")
-public class PublishKafka_2_0 extends AbstractProcessor {
+public class PublishKafka_2_0 extends AbstractProcessor implements KafkaPublishComponent {
     protected static final String MSG_COUNT = "msg.count";
 
     static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("all", "Guarantee Replicated Delivery",
@@ -107,10 +112,6 @@ public class PublishKafka_2_0 extends AbstractProcessor {
         "Interprets the <Partition> property as Expression Language that will be evaluated against each FlowFile. This Expression will be evaluated once against the FlowFile, " +
             "so all Records in a given FlowFile will go to the same partition.");
 
-    static final AllowableValue UTF8_ENCODING = new AllowableValue("utf-8", "UTF-8 Encoded", "The key is interpreted as a UTF-8 Encoded string.");
-    static final AllowableValue HEX_ENCODING = new AllowableValue("hex", "Hex Encoded",
-        "The key is interpreted as arbitrary binary data that is encoded using hexadecimal characters with uppercase letters.");
-
     static final PropertyDescriptor TOPIC = new Builder()
         .name("topic")
         .displayName("Topic Name")
@@ -177,10 +178,10 @@ public class PublishKafka_2_0 extends AbstractProcessor {
     static final PropertyDescriptor KEY_ATTRIBUTE_ENCODING = new Builder()
         .name("key-attribute-encoding")
         .displayName("Key Attribute Encoding")
-        .description("FlowFiles that are emitted have an attribute named '" + KafkaProcessorUtils.KAFKA_KEY + "'. This property dictates how the value of the attribute should be encoded.")
+        .description("FlowFiles that are emitted have an attribute named '" + KafkaFlowFileAttribute.KAFKA_KEY + "'. This property dictates how the value of the attribute should be encoded.")
         .required(true)
-        .defaultValue(UTF8_ENCODING.getValue())
-        .allowableValues(UTF8_ENCODING, HEX_ENCODING)
+        .defaultValue(KeyEncoding.UTF8.getValue())
+        .allowableValues(KeyEncoding.class)
         .build();
 
     static final PropertyDescriptor MESSAGE_DEMARCATOR = new Builder()
@@ -280,7 +281,17 @@ public class PublishKafka_2_0 extends AbstractProcessor {
 
     static {
         final List<PropertyDescriptor> properties = new ArrayList<>();
-        properties.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors());
+        properties.add(BOOTSTRAP_SERVERS);
+        properties.add(SECURITY_PROTOCOL);
+        properties.add(SASL_MECHANISM);
+        properties.add(KERBEROS_SERVICE_NAME);
+        properties.add(KERBEROS_CREDENTIALS_SERVICE);
+        properties.add(KERBEROS_PRINCIPAL);
+        properties.add(KERBEROS_KEYTAB);
+        properties.add(SASL_USERNAME);
+        properties.add(SASL_PASSWORD);
+        properties.add(TOKEN_AUTHENTICATION);
+        properties.add(SSL_CONTEXT_SERVICE);
         properties.add(TOPIC);
         properties.add(DELIVERY_GUARANTEE);
         properties.add(FAILURE_STRATEGY);
@@ -321,7 +332,7 @@ public class PublishKafka_2_0 extends AbstractProcessor {
         return new Builder()
             .description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.")
             .name(propertyDescriptorName)
-            .addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class))
+            .addValidator(new DynamicPropertyValidator(ProducerConfig.class))
             .dynamic(true)
             .expressionLanguageSupported(VARIABLE_REGISTRY)
             .build();
@@ -329,8 +340,7 @@ public class PublishKafka_2_0 extends AbstractProcessor {
 
     @Override
     protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
-        final List<ValidationResult> results = new ArrayList<>();
-        results.addAll(KafkaProcessorUtils.validateCommonProperties(validationContext));
+        final List<ValidationResult> results = new ArrayList<>(new KafkaClientCustomValidationFunction().apply(validationContext));
 
         final boolean useTransactions = validationContext.getProperty(USE_TRANSACTIONS).asBoolean();
         if (useTransactions) {
@@ -371,19 +381,19 @@ public class PublishKafka_2_0 extends AbstractProcessor {
 
     protected PublisherPool createPublisherPool(final ProcessContext context) {
         final int maxMessageSize = context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue();
-        final long maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS).longValue();
+        final long maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
 
         final String attributeNameRegex = context.getProperty(ATTRIBUTE_NAME_REGEX).getValue();
         final Pattern attributeNamePattern = attributeNameRegex == null ? null : Pattern.compile(attributeNameRegex);
         final boolean useTransactions = context.getProperty(USE_TRANSACTIONS).asBoolean();
         final String transactionalIdPrefix = context.getProperty(TRANSACTIONAL_ID_PREFIX).evaluateAttributeExpressions().getValue();
-        Supplier<String> transactionalIdSupplier = KafkaProcessorUtils.getTransactionalIdSupplier(transactionalIdPrefix);
+        Supplier<String> transactionalIdSupplier = new TransactionIdSupplier(transactionalIdPrefix);
 
         final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
         final Charset charset = Charset.forName(charsetName);
 
-        final Map<String, Object> kafkaProperties = new HashMap<>();
-        KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
+        final KafkaPropertyProvider propertyProvider = new StandardKafkaPropertyProvider(ProducerConfig.class);
+        final Map<String, Object> kafkaProperties = propertyProvider.getProperties(context);
         kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize));
@@ -415,8 +425,8 @@ public class PublishKafka_2_0 extends AbstractProcessor {
             return;
         }
 
-        final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue();
-        final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
+        final String securityProtocol = context.getProperty(SECURITY_PROTOCOL).getValue();
+        final String bootstrapServers = context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
         final boolean useTransactions = context.getProperty(USE_TRANSACTIONS).asBoolean();
         final PublishFailureStrategy failureStrategy = getFailureStrategy(context);
 
@@ -451,12 +461,9 @@ public class PublishKafka_2_0 extends AbstractProcessor {
                     }
 
                     final Integer partition = getPartition(context, flowFile);
-                    session.read(flowFile, new InputStreamCallback() {
-                        @Override
-                        public void process(final InputStream rawIn) throws IOException {
-                            try (final InputStream in = new BufferedInputStream(rawIn)) {
-                                lease.publish(flowFile, in, messageKey, demarcatorBytes, topic, partition);
-                            }
+                    session.read(flowFile, rawIn -> {
+                        try (final InputStream in = new BufferedInputStream(rawIn)) {
+                            lease.publish(flowFile, in, messageKey, demarcatorBytes, topic, partition);
                         }
                     });
                 }
@@ -479,7 +486,7 @@ public class PublishKafka_2_0 extends AbstractProcessor {
                     success = session.putAttribute(success, MSG_COUNT, String.valueOf(msgCount));
                     session.adjustCounter("Messages Sent", msgCount, true);
 
-                    final String transitUri = KafkaProcessorUtils.buildTransitURI(securityProtocol, bootstrapServers, topic);
+                    final String transitUri = StandardTransitUriProvider.getTransitUri(securityProtocol, bootstrapServers, topic);
                     session.getProvenanceReporter().send(success, transitUri, "Sent " + msgCount + " messages", transmissionMillis);
                     session.transfer(success, REL_SUCCESS);
                 }
@@ -494,7 +501,7 @@ public class PublishKafka_2_0 extends AbstractProcessor {
 
     private PublishFailureStrategy getFailureStrategy(final ProcessContext context) {
         final String strategy = context.getProperty(FAILURE_STRATEGY).getValue();
-        if (FAILURE_STRATEGY_ROLLBACK.getValue().equals(strategy)) {
+        if (FailureStrategy.ROLLBACK.getValue().equals(strategy)) {
             return (session, flowFiles) -> session.rollback();
         } else {
             return (session, flowFiles) -> session.transfer(flowFiles, REL_FAILURE);
@@ -507,7 +514,7 @@ public class PublishKafka_2_0 extends AbstractProcessor {
         if (context.getProperty(KEY).isSet()) {
             uninterpretedKey = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
         } else {
-            uninterpretedKey = flowFile.getAttribute(KafkaProcessorUtils.KAFKA_KEY);
+            uninterpretedKey = flowFile.getAttribute(KafkaFlowFileAttribute.KAFKA_KEY);
         }
 
         if (uninterpretedKey == null) {
@@ -515,7 +522,7 @@ public class PublishKafka_2_0 extends AbstractProcessor {
         }
 
         final String keyEncoding = context.getProperty(KEY_ATTRIBUTE_ENCODING).getValue();
-        if (UTF8_ENCODING.getValue().equals(keyEncoding)) {
+        if (KeyEncoding.UTF8.getValue().equals(keyEncoding)) {
             return uninterpretedKey.getBytes(StandardCharsets.UTF_8);
         }
 
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/record/sink/kafka/KafkaRecordSink_2_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/record/sink/kafka/KafkaRecordSink_2_0.java
index dba0c61043..53f010ced8 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/record/sink/kafka/KafkaRecordSink_2_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/record/sink/kafka/KafkaRecordSink_2_0.java
@@ -16,14 +16,10 @@
  */
 package org.apache.nifi.record.sink.kafka;
 
-import org.apache.commons.lang3.StringUtils;
-import org.apache.kafka.clients.CommonClientConfigs;
 import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.config.SaslConfigs;
-import org.apache.kafka.common.config.SslConfigs;
 import org.apache.kafka.common.serialization.ByteArraySerializer;
 import org.apache.nifi.annotation.behavior.DynamicProperty;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
@@ -39,10 +35,13 @@ import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ControllerServiceInitializationContext;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.kerberos.KerberosCredentialsService;
+import org.apache.nifi.kafka.shared.component.KafkaPublishComponent;
+import org.apache.nifi.kafka.shared.property.provider.KafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.property.provider.StandardKafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.validation.DynamicPropertyValidator;
+import org.apache.nifi.kafka.shared.validation.KafkaClientCustomValidationFunction;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils;
 import org.apache.nifi.record.sink.RecordSinkService;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.serialization.RecordSetWriter;
@@ -51,18 +50,14 @@ import org.apache.nifi.serialization.WriteResult;
 import org.apache.nifi.serialization.record.Record;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.RecordSet;
-import org.apache.nifi.ssl.SSLContextService;
 import org.apache.nifi.stream.io.ByteCountingOutputStream;
 import org.apache.nifi.stream.io.exception.TokenTooLargeException;
-import org.apache.nifi.util.FormatUtils;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
@@ -76,7 +71,7 @@ import java.util.concurrent.TimeoutException;
                 + " In the event a dynamic property represents a property that was already set, its value will be ignored and WARN message logged."
                 + " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration. ",
         expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY)
-public class KafkaRecordSink_2_0 extends AbstractControllerService implements RecordSinkService {
+public class KafkaRecordSink_2_0 extends AbstractControllerService implements RecordSinkService, KafkaPublishComponent {
 
     static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("all", "Guarantee Replicated Delivery",
             "Records are considered 'transmitted unsuccessfully' unless the message is replicated to the appropriate "
@@ -89,10 +84,6 @@ public class KafkaRecordSink_2_0 extends AbstractControllerService implements Re
             "Records are considered 'transmitted successfully' after successfully writing the content to a Kafka node, "
                     + "without waiting for a response. This provides the best performance but may result in data loss.");
 
-    static final AllowableValue UTF8_ENCODING = new AllowableValue("utf-8", "UTF-8 Encoded", "The key is interpreted as a UTF-8 Encoded string.");
-    static final AllowableValue HEX_ENCODING = new AllowableValue("hex", "Hex Encoded",
-            "The key is interpreted as arbitrary binary data that is encoded using hexadecimal characters with uppercase letters.");
-
     static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder()
             .name("topic")
             .displayName("Topic Name")
@@ -171,17 +162,18 @@ public class KafkaRecordSink_2_0 extends AbstractControllerService implements Re
     private volatile Producer<byte[], byte[]> producer;
 
     @Override
-    protected void init(final ControllerServiceInitializationContext context) throws InitializationException {
+    protected void init(final ControllerServiceInitializationContext context) {
         final List<PropertyDescriptor> properties = new ArrayList<>();
-        properties.add(KafkaProcessorUtils.BOOTSTRAP_SERVERS);
+        properties.add(BOOTSTRAP_SERVERS);
         properties.add(TOPIC);
         properties.add(RecordSinkService.RECORD_WRITER_FACTORY);
         properties.add(DELIVERY_GUARANTEE);
         properties.add(MESSAGE_HEADER_ENCODING);
-        properties.add(KafkaProcessorUtils.SECURITY_PROTOCOL);
-        properties.add(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE);
-        properties.add(KafkaProcessorUtils.JAAS_SERVICE_NAME);
-        properties.add(KafkaProcessorUtils.SSL_CONTEXT_SERVICE);
+        properties.add(SECURITY_PROTOCOL);
+        properties.add(SASL_MECHANISM);
+        properties.add(KERBEROS_CREDENTIALS_SERVICE);
+        properties.add(KERBEROS_SERVICE_NAME);
+        properties.add(SSL_CONTEXT_SERVICE);
         properties.add(MAX_REQUEST_SIZE);
         properties.add(ACK_WAIT_TIME);
         properties.add(METADATA_WAIT_TIME);
@@ -199,7 +191,7 @@ public class KafkaRecordSink_2_0 extends AbstractControllerService implements Re
         return new PropertyDescriptor.Builder()
                 .description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.")
                 .name(propertyDescriptorName)
-                .addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class))
+                .addValidator(new DynamicPropertyValidator(ProducerConfig.class))
                 .dynamic(true)
                 .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
                 .build();
@@ -207,7 +199,7 @@ public class KafkaRecordSink_2_0 extends AbstractControllerService implements Re
 
     @Override
     protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
-        return KafkaProcessorUtils.validateCommonProperties(validationContext);
+        return new KafkaClientCustomValidationFunction().apply(validationContext);
     }
 
     @OnEnabled
@@ -217,11 +209,8 @@ public class KafkaRecordSink_2_0 extends AbstractControllerService implements Re
         maxMessageSize = context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue();
         maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
 
-        final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
-        final Charset charset = Charset.forName(charsetName);
-
-        final Map<String, Object> kafkaProperties = new HashMap<>();
-        buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
+        final KafkaPropertyProvider propertyProvider = new StandardKafkaPropertyProvider(ProducerConfig.class);
+        final Map<String, Object> kafkaProperties = propertyProvider.getProperties(context);
         kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize));
@@ -299,93 +288,12 @@ public class KafkaRecordSink_2_0 extends AbstractControllerService implements Re
     }
 
     @OnDisabled
-    public void stop() throws IOException {
+    public void stop() {
         if (producer != null) {
             producer.close(maxAckWaitMillis, TimeUnit.MILLISECONDS);
         }
     }
 
-    static void buildCommonKafkaProperties(final ConfigurationContext context, final Class<?> kafkaConfigClass, final Map<String, Object> mapToPopulate) {
-        for (PropertyDescriptor propertyDescriptor : context.getProperties().keySet()) {
-            if (propertyDescriptor.equals(KafkaProcessorUtils.SSL_CONTEXT_SERVICE)) {
-                // Translate SSLContext Service configuration into Kafka properties
-                final SSLContextService sslContextService = context.getProperty(KafkaProcessorUtils.SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
-                if (sslContextService != null && sslContextService.isKeyStoreConfigured()) {
-                    mapToPopulate.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, sslContextService.getKeyStoreFile());
-                    mapToPopulate.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, sslContextService.getKeyStorePassword());
-                    final String keyPass = sslContextService.getKeyPassword() == null ? sslContextService.getKeyStorePassword() : sslContextService.getKeyPassword();
-                    mapToPopulate.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, keyPass);
-                    mapToPopulate.put(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, sslContextService.getKeyStoreType());
-                }
-
-                if (sslContextService != null && sslContextService.isTrustStoreConfigured()) {
-                    mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, sslContextService.getTrustStoreFile());
-                    mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, sslContextService.getTrustStorePassword());
-                    mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, sslContextService.getTrustStoreType());
-                }
-            }
-
-            String propertyName = propertyDescriptor.getName();
-            String propertyValue = propertyDescriptor.isExpressionLanguageSupported()
-                    ? context.getProperty(propertyDescriptor).evaluateAttributeExpressions().getValue()
-                    : context.getProperty(propertyDescriptor).getValue();
-
-            if (propertyValue != null) {
-                // If the property name ends in ".ms" then it is a time period. We want to accept either an integer as number of milliseconds
-                // or the standard NiFi time period such as "5 secs"
-                if (propertyName.endsWith(".ms") && !StringUtils.isNumeric(propertyValue.trim())) { // kafka standard time notation
-                    propertyValue = String.valueOf(FormatUtils.getTimeDuration(propertyValue.trim(), TimeUnit.MILLISECONDS));
-                }
-
-                if (KafkaProcessorUtils.isStaticStringFieldNamePresent(propertyName, kafkaConfigClass, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class)) {
-                    mapToPopulate.put(propertyName, propertyValue);
-                }
-            }
-        }
-
-        String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue();
-        if (KafkaProcessorUtils.SEC_SASL_PLAINTEXT.getValue().equals(securityProtocol) || KafkaProcessorUtils.SEC_SASL_SSL.getValue().equals(securityProtocol)) {
-            setJaasConfig(mapToPopulate, context);
-        }
-    }
-
-    /**
-     * Method used to configure the 'sasl.jaas.config' property based on KAFKA-4259<br />
-     * https://cwiki.apache.org/confluence/display/KAFKA/KIP-85%3A+Dynamic+JAAS+configuration+for+Kafka+clients<br />
-     * <br />
-     * It expects something with the following format: <br />
-     * <br />
-     * &lt;LoginModuleClass&gt; &lt;ControlFlag&gt; *(&lt;OptionName&gt;=&lt;OptionValue&gt;); <br />
-     * ControlFlag = required / requisite / sufficient / optional
-     *
-     * @param mapToPopulate Map of configuration properties
-     * @param context       Context
-     */
-    private static void setJaasConfig(Map<String, Object> mapToPopulate, ConfigurationContext context) {
-        String keytab = null;
-        String principal = null;
-
-        // If the Kerberos Credentials Service is specified, we need to use its configuration, not the explicit properties for principal/keytab.
-        // The customValidate method ensures that only one can be set, so we know that the principal & keytab above are null.
-        final KerberosCredentialsService credentialsService = context.getProperty(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
-        if (credentialsService != null) {
-            principal = credentialsService.getPrincipal();
-            keytab = credentialsService.getKeytab();
-        }
-
-
-        String serviceName = context.getProperty(KafkaProcessorUtils.JAAS_SERVICE_NAME).evaluateAttributeExpressions().getValue();
-        if (StringUtils.isNotBlank(keytab) && StringUtils.isNotBlank(principal) && StringUtils.isNotBlank(serviceName)) {
-            mapToPopulate.put(SaslConfigs.SASL_JAAS_CONFIG, "com.sun.security.auth.module.Krb5LoginModule required "
-                    + "useTicketCache=false "
-                    + "renewTicket=true "
-                    + "serviceName=\"" + serviceName + "\" "
-                    + "useKeyTab=true "
-                    + "keyTab=\"" + keytab + "\" "
-                    + "principal=\"" + principal + "\";");
-        }
-    }
-
     // this getter is intended explicitly for testing purposes
     protected RecordSetWriterFactory getWriterFactory() {
         return this.writerFactory;
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ITConsumeKafka_2_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ITConsumeKafka_2_0.java
index c4e8813d8f..ae1c03dd6e 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ITConsumeKafka_2_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ITConsumeKafka_2_0.java
@@ -42,7 +42,7 @@ public class ITConsumeKafka_2_0 {
     }
 
     @Test
-    public void validateGetAllMessages() throws Exception {
+    public void validateGetAllMessages() {
         String groupName = "validateGetAllMessages";
 
         when(mockConsumerPool.obtainConsumer(any(), any())).thenReturn(mockLease);
@@ -56,7 +56,7 @@ public class ITConsumeKafka_2_0 {
             }
         };
         final TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
+        runner.setProperty(ConsumeKafka_2_0.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
         runner.setProperty(ConsumeKafka_2_0.TOPICS, "foo,bar");
         runner.setProperty(ConsumeKafka_2_0.GROUP_ID, groupName);
         runner.setProperty(ConsumeKafka_2_0.AUTO_OFFSET_RESET, ConsumeKafka_2_0.OFFSET_EARLIEST);
@@ -72,7 +72,7 @@ public class ITConsumeKafka_2_0 {
     }
 
     @Test
-    public void validateGetAllMessagesPattern() throws Exception {
+    public void validateGetAllMessagesPattern() {
         String groupName = "validateGetAllMessagesPattern";
 
         when(mockConsumerPool.obtainConsumer(any(), any())).thenReturn(mockLease);
@@ -86,7 +86,7 @@ public class ITConsumeKafka_2_0 {
             }
         };
         final TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
+        runner.setProperty(ConsumeKafka_2_0.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
         runner.setProperty(ConsumeKafka_2_0.TOPICS, "(fo.*)|(ba)");
         runner.setProperty(ConsumeKafka_2_0.TOPIC_TYPE, "pattern");
         runner.setProperty(ConsumeKafka_2_0.GROUP_ID, groupName);
@@ -103,7 +103,7 @@ public class ITConsumeKafka_2_0 {
     }
 
     @Test
-    public void validateGetErrorMessages() throws Exception {
+    public void validateGetErrorMessages() {
         String groupName = "validateGetErrorMessages";
 
         when(mockConsumerPool.obtainConsumer(any(), any())).thenReturn(mockLease);
@@ -117,7 +117,7 @@ public class ITConsumeKafka_2_0 {
             }
         };
         final TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
+        runner.setProperty(ConsumeKafka_2_0.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
         runner.setProperty(ConsumeKafka_2_0.TOPICS, "foo,bar");
         runner.setProperty(ConsumeKafka_2_0.GROUP_ID, groupName);
         runner.setProperty(ConsumeKafka_2_0.AUTO_OFFSET_RESET, ConsumeKafka_2_0.OFFSET_EARLIEST);
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumeKafkaRecord_2_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumeKafkaRecord_2_0.java
index e57d0705b8..2275cbdba7 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumeKafkaRecord_2_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumeKafkaRecord_2_0.java
@@ -18,6 +18,8 @@ package org.apache.nifi.processors.kafka.pubsub;
 
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.nifi.kafka.shared.property.SaslMechanism;
+import org.apache.nifi.kafka.shared.property.SecurityProtocol;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processors.kafka.pubsub.util.MockRecordParser;
@@ -58,7 +60,7 @@ public class TestConsumeKafkaRecord_2_0 {
         };
 
         runner = TestRunners.newTestRunner(proc);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
+        runner.setProperty(ConsumeKafkaRecord_2_0.BOOTSTRAP_SERVERS, "okeydokey:1234");
 
         final String readerId = "record-reader";
         final MockRecordParser readerService = new MockRecordParser();
@@ -77,24 +79,18 @@ public class TestConsumeKafkaRecord_2_0 {
     }
 
     @Test
-    public void validateCustomValidatorSettings() throws Exception {
+    public void validateCustomValidatorSettings() {
         runner.setProperty(ConsumeKafkaRecord_2_0.TOPICS, "foo");
         runner.setProperty(ConsumeKafkaRecord_2_0.GROUP_ID, "foo");
         runner.setProperty(ConsumeKafkaRecord_2_0.AUTO_OFFSET_RESET, ConsumeKafkaRecord_2_0.OFFSET_EARLIEST);
         runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
         runner.assertValid();
-        runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "Foo");
-        runner.assertNotValid();
-        runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
-        runner.assertValid();
         runner.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
         runner.assertValid();
-        runner.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true");
-        runner.assertNotValid();
     }
 
     @Test
-    public void validatePropertiesValidation() throws Exception {
+    public void validatePropertiesValidation() {
         runner.setProperty(ConsumeKafkaRecord_2_0.TOPICS, "foo");
         runner.setProperty(ConsumeKafkaRecord_2_0.GROUP_ID, "foo");
         runner.setProperty(ConsumeKafkaRecord_2_0.AUTO_OFFSET_RESET, ConsumeKafkaRecord_2_0.OFFSET_EARLIEST);
@@ -116,7 +112,7 @@ public class TestConsumeKafkaRecord_2_0 {
     }
 
     @Test
-    public void validateGetAllMessages() throws Exception {
+    public void validateGetAllMessages() {
         String groupName = "validateGetAllMessages";
 
         when(mockConsumerPool.obtainConsumer(any(), any())).thenReturn(mockLease);
@@ -138,7 +134,7 @@ public class TestConsumeKafkaRecord_2_0 {
     }
 
     @Test
-    public void validateGetAllMessagesPattern() throws Exception {
+    public void validateGetAllMessagesPattern() {
         String groupName = "validateGetAllMessagesPattern";
 
         when(mockConsumerPool.obtainConsumer(any(), any())).thenReturn(mockLease);
@@ -161,7 +157,7 @@ public class TestConsumeKafkaRecord_2_0 {
     }
 
     @Test
-    public void validateGetErrorMessages() throws Exception {
+    public void validateGetErrorMessages() {
         String groupName = "validateGetErrorMessages";
 
         when(mockConsumerPool.obtainConsumer(any(), any())).thenReturn(mockLease);
@@ -188,19 +184,19 @@ public class TestConsumeKafkaRecord_2_0 {
         runner.setProperty(ConsumeKafkaRecord_2_0.GROUP_ID, "foo");
         runner.setProperty(ConsumeKafkaRecord_2_0.AUTO_OFFSET_RESET, ConsumeKafkaRecord_2_0.OFFSET_EARLIEST);
 
-        runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT);
+        runner.setProperty(ConsumeKafkaRecord_2_0.SECURITY_PROTOCOL, SecurityProtocol.SASL_PLAINTEXT.name());
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.JAAS_SERVICE_NAME, "kafka");
-        runner.assertValid();
+        runner.setProperty(ConsumeKafkaRecord_2_0.KERBEROS_SERVICE_NAME, "kafka");
+        runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.USER_PRINCIPAL, "nifi@APACHE.COM");
+        runner.setProperty(ConsumeKafkaRecord_2_0.KERBEROS_PRINCIPAL, "nifi@APACHE.COM");
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "not.A.File");
+        runner.setProperty(ConsumeKafkaRecord_2_0.KERBEROS_KEYTAB, "not.A.File");
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "src/test/resources/server.properties");
+        runner.setProperty(ConsumeKafkaRecord_2_0.KERBEROS_KEYTAB, "src/test/resources/server.properties");
         runner.assertValid();
     }
 
@@ -210,19 +206,19 @@ public class TestConsumeKafkaRecord_2_0 {
         runner.setProperty(ConsumeKafkaRecord_2_0.GROUP_ID, "foo");
         runner.setProperty(ConsumeKafkaRecord_2_0.AUTO_OFFSET_RESET, ConsumeKafkaRecord_2_0.OFFSET_EARLIEST);
 
-        runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT);
+        runner.setProperty(ConsumeKafkaRecord_2_0.SECURITY_PROTOCOL, SecurityProtocol.SASL_PLAINTEXT.name());
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.SASL_MECHANISM, KafkaProcessorUtils.PLAIN_VALUE);
+        runner.setProperty(ConsumeKafkaRecord_2_0.SASL_MECHANISM, SaslMechanism.PLAIN.getValue());
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.USERNAME, "user1");
+        runner.setProperty(ConsumeKafkaRecord_2_0.SASL_USERNAME, "user1");
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.PASSWORD, "password");
+        runner.setProperty(ConsumeKafkaRecord_2_0.SASL_PASSWORD, "password");
         runner.assertValid();
 
-        runner.removeProperty(KafkaProcessorUtils.USERNAME);
+        runner.removeProperty(ConsumeKafkaRecord_2_0.SASL_USERNAME);
         runner.assertNotValid();
     }
 
@@ -232,19 +228,19 @@ public class TestConsumeKafkaRecord_2_0 {
         runner.setProperty(ConsumeKafkaRecord_2_0.GROUP_ID, "foo");
         runner.setProperty(ConsumeKafkaRecord_2_0.AUTO_OFFSET_RESET, ConsumeKafkaRecord_2_0.OFFSET_EARLIEST);
 
-        runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT);
+        runner.setProperty(ConsumeKafkaRecord_2_0.SECURITY_PROTOCOL, SecurityProtocol.SASL_PLAINTEXT.name());
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.SASL_MECHANISM, KafkaProcessorUtils.SCRAM_SHA256_VALUE);
+        runner.setProperty(ConsumeKafkaRecord_2_0.SASL_MECHANISM, SaslMechanism.SCRAM_SHA_256.getValue());
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.USERNAME, "user1");
+        runner.setProperty(ConsumeKafkaRecord_2_0.SASL_USERNAME, "user1");
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.PASSWORD, "password");
+        runner.setProperty(ConsumeKafkaRecord_2_0.SASL_PASSWORD, "password");
         runner.assertValid();
 
-        runner.removeProperty(KafkaProcessorUtils.USERNAME);
+        runner.removeProperty(ConsumeKafkaRecord_2_0.SASL_USERNAME);
         runner.assertNotValid();
     }
 
@@ -254,19 +250,19 @@ public class TestConsumeKafkaRecord_2_0 {
         runner.setProperty(ConsumeKafkaRecord_2_0.GROUP_ID, "foo");
         runner.setProperty(ConsumeKafkaRecord_2_0.AUTO_OFFSET_RESET, ConsumeKafkaRecord_2_0.OFFSET_EARLIEST);
 
-        runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT);
+        runner.setProperty(ConsumeKafkaRecord_2_0.SECURITY_PROTOCOL, SecurityProtocol.SASL_PLAINTEXT.name());
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.SASL_MECHANISM, KafkaProcessorUtils.SCRAM_SHA512_VALUE);
+        runner.setProperty(ConsumeKafkaRecord_2_0.SASL_MECHANISM, SaslMechanism.SCRAM_SHA_512.getValue());
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.USERNAME, "user1");
+        runner.setProperty(ConsumeKafkaRecord_2_0.SASL_USERNAME, "user1");
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.PASSWORD, "password");
+        runner.setProperty(ConsumeKafkaRecord_2_0.SASL_PASSWORD, "password");
         runner.assertValid();
 
-        runner.removeProperty(KafkaProcessorUtils.USERNAME);
+        runner.removeProperty(ConsumeKafkaRecord_2_0.SASL_USERNAME);
         runner.assertNotValid();
     }
 
@@ -276,7 +272,7 @@ public class TestConsumeKafkaRecord_2_0 {
         runner.setProperty(ConsumeKafkaRecord_2_0.GROUP_ID, "foo");
         runner.setProperty(ConsumeKafkaRecord_2_0.AUTO_OFFSET_RESET, ConsumeKafkaRecord_2_0.OFFSET_EARLIEST);
 
-        runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_PLAINTEXT);
+        runner.setProperty(ConsumeKafkaRecord_2_0.SECURITY_PROTOCOL, SecurityProtocol.PLAINTEXT.name());
         runner.assertValid();
     }
 
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumeKafka_2_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumeKafka_2_0.java
index 6d5f44f202..560af84170 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumeKafka_2_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumeKafka_2_0.java
@@ -18,6 +18,8 @@ package org.apache.nifi.processors.kafka.pubsub;
 
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.nifi.kafka.shared.property.SaslMechanism;
+import org.apache.nifi.kafka.shared.property.SecurityProtocol;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.jupiter.api.BeforeEach;
@@ -39,80 +41,77 @@ public class TestConsumeKafka_2_0 {
     }
 
     @Test
-    public void validateCustomValidatorSettings() throws Exception {
+    public void validateCustomValidatorSettings() {
         ConsumeKafka_2_0 consumeKafka = new ConsumeKafka_2_0();
         TestRunner runner = TestRunners.newTestRunner(consumeKafka);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
+        runner.setProperty(ConsumeKafka_2_0.BOOTSTRAP_SERVERS, "okeydokey:1234");
         runner.setProperty(ConsumeKafka_2_0.TOPICS, "foo");
         runner.setProperty(ConsumeKafka_2_0.GROUP_ID, "foo");
         runner.setProperty(ConsumeKafka_2_0.AUTO_OFFSET_RESET, ConsumeKafka_2_0.OFFSET_EARLIEST);
         runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
         runner.assertValid();
-        runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "Foo");
-        runner.assertNotValid();
         runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
         runner.assertValid();
         runner.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
         runner.assertValid();
-        runner.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true");
-        runner.assertNotValid();
     }
 
     @Test
-    public void validatePropertiesValidation() throws Exception {
+    public void validatePropertiesValidation() {
         ConsumeKafka_2_0 consumeKafka = new ConsumeKafka_2_0();
         TestRunner runner = TestRunners.newTestRunner(consumeKafka);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
+        runner.setProperty(ConsumeKafka_2_0.BOOTSTRAP_SERVERS, "okeydokey:1234");
         runner.setProperty(ConsumeKafka_2_0.TOPICS, "foo");
         runner.setProperty(ConsumeKafka_2_0.GROUP_ID, "foo");
         runner.setProperty(ConsumeKafka_2_0.AUTO_OFFSET_RESET, ConsumeKafka_2_0.OFFSET_EARLIEST);
 
         runner.removeProperty(ConsumeKafka_2_0.GROUP_ID);
 
-        AssertionError e = assertThrows(AssertionError.class, () -> runner.assertValid());
+        AssertionError e = assertThrows(AssertionError.class, runner::assertValid);
         assertTrue(e.getMessage().contains("invalid because Group ID is required"));
 
         runner.setProperty(ConsumeKafka_2_0.GROUP_ID, "");
 
-        e = assertThrows(AssertionError.class, () -> runner.assertValid());
+        e = assertThrows(AssertionError.class, runner::assertValid);
         assertTrue(e.getMessage().contains("must contain at least one character that is not white space"));
 
         runner.setProperty(ConsumeKafka_2_0.GROUP_ID, "  ");
 
-        e = assertThrows(AssertionError.class, () -> runner.assertValid());
+        e = assertThrows(AssertionError.class, runner::assertValid);
         assertTrue(e.getMessage().contains("must contain at least one character that is not white space"));
     }
 
     @Test
-    public void testJaasConfiguration() throws Exception {
+    public void testJaasConfiguration() {
         ConsumeKafka_2_0 consumeKafka = new ConsumeKafka_2_0();
         TestRunner runner = TestRunners.newTestRunner(consumeKafka);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
+        runner.setProperty(ConsumeKafka_2_0.BOOTSTRAP_SERVERS, "okeydokey:1234");
         runner.setProperty(ConsumeKafka_2_0.TOPICS, "foo");
         runner.setProperty(ConsumeKafka_2_0.GROUP_ID, "foo");
         runner.setProperty(ConsumeKafka_2_0.AUTO_OFFSET_RESET, ConsumeKafka_2_0.OFFSET_EARLIEST);
 
-        runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT);
+        runner.setProperty(ConsumeKafka_2_0.SECURITY_PROTOCOL, SecurityProtocol.SASL_PLAINTEXT.name());
+        runner.setProperty(ConsumeKafka_2_0.SASL_MECHANISM, SaslMechanism.GSSAPI.getValue());
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.JAAS_SERVICE_NAME, "kafka");
-        runner.assertValid();
+        runner.setProperty(ConsumeKafka_2_0.KERBEROS_SERVICE_NAME, "kafka");
+        runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.USER_PRINCIPAL, "nifi@APACHE.COM");
+        runner.setProperty(ConsumeKafka_2_0.KERBEROS_PRINCIPAL, "nifi@APACHE.COM");
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "not.A.File");
+        runner.setProperty(ConsumeKafka_2_0.KERBEROS_KEYTAB, "not.A.File");
         runner.assertNotValid();
 
-        runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "src/test/resources/server.properties");
+        runner.setProperty(ConsumeKafka_2_0.KERBEROS_KEYTAB, "src/test/resources/server.properties");
         runner.assertValid();
 
         runner.setVariable("keytab", "src/test/resources/server.properties");
         runner.setVariable("principal", "nifi@APACHE.COM");
         runner.setVariable("service", "kafka");
-        runner.setProperty(KafkaProcessorUtils.USER_PRINCIPAL, "${principal}");
-        runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "${keytab}");
-        runner.setProperty(KafkaProcessorUtils.JAAS_SERVICE_NAME, "${service}");
+        runner.setProperty(ConsumeKafka_2_0.KERBEROS_PRINCIPAL, "${principal}");
+        runner.setProperty(ConsumeKafka_2_0.KERBEROS_KEYTAB, "${keytab}");
+        runner.setProperty(ConsumeKafka_2_0.KERBEROS_SERVICE_NAME, "${service}");
         runner.assertValid();
     }
 
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumerPartitionsUtil.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumerPartitionsUtil.java
index 4f53bab922..b70dd41a81 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumerPartitionsUtil.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumerPartitionsUtil.java
@@ -42,7 +42,7 @@ public class TestConsumerPartitionsUtil {
 
     @BeforeEach
     public void setup() throws UnknownHostException {
-        hostname = InetAddress.getLocalHost().getHostName();;
+        hostname = InetAddress.getLocalHost().getHostName();
     }
 
     @Test
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaRecord_2_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaRecord_2_0.java
index 0feed620df..b3110337de 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaRecord_2_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaRecord_2_0.java
@@ -19,6 +19,7 @@ package org.apache.nifi.processors.kafka.pubsub;
 
 import org.apache.kafka.common.errors.ProducerFencedException;
 import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.kafka.shared.property.FailureStrategy;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processors.kafka.pubsub.util.MockRecordParser;
 import org.apache.nifi.reporting.InitializationException;
@@ -155,7 +156,7 @@ public class TestPublishKafkaRecord_2_0 {
 
     @Test
     public void testSingleFailureWithRollback() throws IOException {
-        runner.setProperty(KafkaProcessorUtils.FAILURE_STRATEGY, KafkaProcessorUtils.FAILURE_STRATEGY_ROLLBACK);
+        runner.setProperty(PublishKafkaRecord_2_0.FAILURE_STRATEGY, FailureStrategy.ROLLBACK.getValue());
 
         final MockFlowFile flowFile = runner.enqueue("John Doe, 48");
 
@@ -189,7 +190,7 @@ public class TestPublishKafkaRecord_2_0 {
 
     @Test
     public void testFailureWhenCreatingTransactionWithRollback() {
-        runner.setProperty(KafkaProcessorUtils.FAILURE_STRATEGY, KafkaProcessorUtils.FAILURE_STRATEGY_ROLLBACK);
+        runner.setProperty(PublishKafkaRecord_2_0.FAILURE_STRATEGY, FailureStrategy.ROLLBACK.getValue());
         runner.enqueue("John Doe, 48");
 
         doAnswer((Answer<Object>) invocationOnMock -> {
@@ -224,7 +225,7 @@ public class TestPublishKafkaRecord_2_0 {
 
     @Test
     public void testMultipleFailuresWithRollback() throws IOException {
-        runner.setProperty(KafkaProcessorUtils.FAILURE_STRATEGY, KafkaProcessorUtils.FAILURE_STRATEGY_ROLLBACK);
+        runner.setProperty(PublishKafkaRecord_2_0.FAILURE_STRATEGY, FailureStrategy.ROLLBACK.getValue());
         final Set<FlowFile> flowFiles = new HashSet<>();
         flowFiles.add(runner.enqueue("John Doe, 48"));
         flowFiles.add(runner.enqueue("John Doe, 48"));
@@ -423,7 +424,7 @@ public class TestPublishKafkaRecord_2_0 {
             @Override
             public int getSuccessfulMessageCount(FlowFile flowFile) {
                 Integer count = msgCounts.get(flowFile);
-                return count == null ? 0 : count.intValue();
+                return count == null ? 0 : count;
             }
 
             @Override
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka_2_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka_2_0.java
index 1433a896d8..92fb25da32 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka_2_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka_2_0.java
@@ -19,6 +19,7 @@ package org.apache.nifi.processors.kafka.pubsub;
 
 import org.apache.kafka.common.errors.ProducerFencedException;
 import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.kafka.shared.property.FailureStrategy;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
@@ -139,7 +140,7 @@ public class TestPublishKafka_2_0 {
 
     @Test
     public void testSingleFailureWithRollback() throws IOException {
-        runner.setProperty(KafkaProcessorUtils.FAILURE_STRATEGY, KafkaProcessorUtils.FAILURE_STRATEGY_ROLLBACK);
+        runner.setProperty(PublishKafka_2_0.FAILURE_STRATEGY, FailureStrategy.ROLLBACK.getValue());
         final MockFlowFile flowFile = runner.enqueue("hello world");
 
         when(mockLease.complete()).thenReturn(createFailurePublishResult(flowFile));
@@ -172,7 +173,7 @@ public class TestPublishKafka_2_0 {
 
     @Test
     public void testMultipleFailuresWithRollback() throws IOException {
-        runner.setProperty(KafkaProcessorUtils.FAILURE_STRATEGY, KafkaProcessorUtils.FAILURE_STRATEGY_ROLLBACK);
+        runner.setProperty(PublishKafka_2_0.FAILURE_STRATEGY, FailureStrategy.ROLLBACK.getValue());
 
         final Set<FlowFile> flowFiles = new HashSet<>();
         flowFiles.add(runner.enqueue("hello world"));
@@ -293,7 +294,7 @@ public class TestPublishKafka_2_0 {
             @Override
             public int getSuccessfulMessageCount(FlowFile flowFile) {
                 Integer count = msgCounts.get(flowFile);
-                return count == null ? 0 : count.intValue();
+                return count == null ? 0 : count;
             }
 
             @Override
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/record/sink/kafka/TestKafkaRecordSink_2_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/record/sink/kafka/TestKafkaRecordSink_2_0.java
index d1bd0ff795..4eb0e08504 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/record/sink/kafka/TestKafkaRecordSink_2_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/record/sink/kafka/TestKafkaRecordSink_2_0.java
@@ -27,10 +27,10 @@ import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ControllerServiceInitializationContext;
+import org.apache.nifi.kafka.shared.property.SecurityProtocol;
 import org.apache.nifi.kerberos.KerberosCredentialsService;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils;
 import org.apache.nifi.record.sink.RecordSinkService;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.serialization.RecordSetWriterFactory;
@@ -45,7 +45,7 @@ import org.apache.nifi.serialization.record.RecordSet;
 import org.apache.nifi.ssl.SSLContextService;
 import org.apache.nifi.state.MockStateManager;
 import org.apache.nifi.util.MockControllerServiceInitializationContext;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.mockito.ArgumentMatcher;
 import org.mockito.Mockito;
 import org.mockito.stubbing.Answer;
@@ -138,13 +138,13 @@ public class TestKafkaRecordSink_2_0 {
         when(context.getProperty(KafkaRecordSink_2_0.MESSAGE_HEADER_ENCODING)).thenReturn(charEncodingValue);
 
         final PropertyValue securityValue = Mockito.mock(StandardPropertyValue.class);
-        when(securityValue.getValue()).thenReturn(KafkaProcessorUtils.SEC_SASL_PLAINTEXT.getValue());
-        when(context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL)).thenReturn(securityValue);
+        when(securityValue.getValue()).thenReturn(SecurityProtocol.PLAINTEXT.name());
+        when(context.getProperty(KafkaRecordSink_2_0.SECURITY_PROTOCOL)).thenReturn(securityValue);
 
         final PropertyValue jaasValue = Mockito.mock(StandardPropertyValue.class);
         when(jaasValue.evaluateAttributeExpressions()).thenReturn(jaasValue);
         when(jaasValue.getValue()).thenReturn(null);
-        when(context.getProperty(KafkaProcessorUtils.JAAS_SERVICE_NAME)).thenReturn(jaasValue);
+        when(context.getProperty(KafkaRecordSink_2_0.KERBEROS_SERVICE_NAME)).thenReturn(jaasValue);
 
         Map<PropertyDescriptor, String> propertyMap = new HashMap<>();
         propertyMap.put(KafkaRecordSink_2_0.TOPIC, KafkaRecordSink_2_0.TOPIC.getName());
@@ -160,9 +160,9 @@ public class TestKafkaRecordSink_2_0 {
         MockRecordWriter writer = new MockRecordWriter(null, false);
         when(context.getProperty(RecordSinkService.RECORD_WRITER_FACTORY)).thenReturn(pValue);
         when(pValue.asControllerService(RecordSetWriterFactory.class)).thenReturn(writer);
-        when(context.getProperty(KafkaProcessorUtils.SSL_CONTEXT_SERVICE)).thenReturn(pValue);
+        when(context.getProperty(KafkaRecordSink_2_0.SSL_CONTEXT_SERVICE)).thenReturn(pValue);
         when(pValue.asControllerService(SSLContextService.class)).thenReturn(null);
-        when(context.getProperty(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE)).thenReturn(pValue);
+        when(context.getProperty(KafkaRecordSink_2_0.KERBEROS_CREDENTIALS_SERVICE)).thenReturn(pValue);
         when(pValue.asControllerService(KerberosCredentialsService.class)).thenReturn(null);
 
         final ControllerServiceInitializationContext initContext = new MockControllerServiceInitializationContext(task, UUID.randomUUID().toString(), logger, stateManager);
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/resources/log4j.properties b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/resources/log4j.properties
deleted file mode 100644
index 57cd63f139..0000000000
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/resources/log4j.properties
+++ /dev/null
@@ -1,21 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#    http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-log4j.rootCategory=INFO, stdout
-
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=%d{ABSOLUTE} %5p %t %c{2}:%L - %m%n
-
-#og4j.category.org.apache.nifi.processors.kafka=DEBUG


[nifi] 11/22: NIFI-10882 Set credentials for ElasticSearchClientService based on AuthorizationScheme

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit d5028315592028b91f793ab5d1d48fee2754075b
Author: Chris Sampson <ch...@gmail.com>
AuthorDate: Mon Nov 28 10:25:19 2022 +0000

    NIFI-10882 Set credentials for ElasticSearchClientService based on AuthorizationScheme
    
    This closes #6722
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 .../nifi/elasticsearch/ElasticSearchClientServiceImpl.java     | 10 ++++++----
 1 file changed, 6 insertions(+), 4 deletions(-)

diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientServiceImpl.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientServiceImpl.java
index 4809b85fd2..ca7c2d306f 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientServiceImpl.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientServiceImpl.java
@@ -249,6 +249,8 @@ public class ElasticSearchClientServiceImpl extends AbstractControllerService im
     }
 
     private RestClient setupClient(final ConfigurationContext context) throws MalformedURLException, InitializationException {
+        final AuthorizationScheme authorizationScheme = AuthorizationScheme.valueOf(context.getProperty(AUTHORIZATION_SCHEME).getValue());
+
         final String hosts = context.getProperty(HTTP_HOSTS).evaluateAttributeExpressions().getValue();
         final String[] hostsSplit = hosts.split(",\\s*");
         this.url = hostsSplit[0];
@@ -261,7 +263,7 @@ public class ElasticSearchClientServiceImpl extends AbstractControllerService im
         final String apiKey = context.getProperty(API_KEY).getValue();
 
         final Integer connectTimeout = context.getProperty(CONNECT_TIMEOUT).asInteger();
-        final Integer readTimeout    = context.getProperty(SOCKET_TIMEOUT).asInteger();
+        final Integer socketTimeout = context.getProperty(SOCKET_TIMEOUT).asInteger();
 
         final ProxyConfigurationService proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE).asControllerService(ProxyConfigurationService.class);
 
@@ -287,11 +289,11 @@ public class ElasticSearchClientServiceImpl extends AbstractControllerService im
                     }
 
                     CredentialsProvider credentialsProvider = null;
-                    if (username != null && password != null) {
+                    if (AuthorizationScheme.BASIC == authorizationScheme && username != null && password != null) {
                         credentialsProvider = addCredentials(null, AuthScope.ANY, username, password);
                     }
 
-                    if (apiKeyId != null && apiKey != null) {
+                    if (AuthorizationScheme.API_KEY == authorizationScheme && apiKeyId != null && apiKey != null) {
                         httpClientBuilder.setDefaultHeaders(Collections.singletonList(createApiKeyAuthorizationHeader(apiKeyId, apiKey)));
                     }
 
@@ -313,7 +315,7 @@ public class ElasticSearchClientServiceImpl extends AbstractControllerService im
                 })
                 .setRequestConfigCallback(requestConfigBuilder -> {
                     requestConfigBuilder.setConnectTimeout(connectTimeout);
-                    requestConfigBuilder.setSocketTimeout(readTimeout);
+                    requestConfigBuilder.setSocketTimeout(socketTimeout);
                     return requestConfigBuilder;
                 });
 


[nifi] 15/22: NIFI-10871 Skipped CSRF processing for replicated HTTP requests

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 4aa86b850db6ed605f1d987127eb08cf15915534
Author: exceptionfactory <ex...@apache.org>
AuthorDate: Wed Nov 23 16:30:20 2022 -0600

    NIFI-10871 Skipped CSRF processing for replicated HTTP requests
    
    - Updated Security Filter Configuration to avoid unnecessary CSRF Request Token generation for requests replicated between cluster nodes
    
    This closes #6715.
    
    Signed-off-by: Tamas Palfy <tp...@apache.org>
---
 .../nifi/web/NiFiWebApiSecurityConfiguration.java  |   2 +
 .../security/csrf/SkipReplicatedCsrfFilter.java    |  61 +++++++++++
 .../csrf/SkipReplicatedCsrfFilterTest.java         | 113 +++++++++++++++++++++
 3 files changed, 176 insertions(+)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiSecurityConfiguration.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiSecurityConfiguration.java
index d43824d122..2108b09abe 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiSecurityConfiguration.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiSecurityConfiguration.java
@@ -20,6 +20,7 @@ import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.security.StandardAuthenticationEntryPoint;
 import org.apache.nifi.web.security.anonymous.NiFiAnonymousAuthenticationFilter;
 import org.apache.nifi.web.security.csrf.CsrfCookieRequestMatcher;
+import org.apache.nifi.web.security.csrf.SkipReplicatedCsrfFilter;
 import org.apache.nifi.web.security.csrf.StandardCookieCsrfTokenRepository;
 import org.apache.nifi.web.security.knox.KnoxAuthenticationFilter;
 import org.apache.nifi.web.security.log.AuthenticationUserFilter;
@@ -109,6 +110,7 @@ public class NiFiWebApiSecurityConfiguration {
                         ).permitAll()
                         .anyRequest().authenticated()
                 )
+                .addFilterBefore(new SkipReplicatedCsrfFilter(), CsrfFilter.class)
                 .csrf(csrf -> csrf
                         .csrfTokenRepository(
                                 new StandardCookieCsrfTokenRepository()
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/csrf/SkipReplicatedCsrfFilter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/csrf/SkipReplicatedCsrfFilter.java
new file mode 100644
index 0000000000..e3637fdb55
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/csrf/SkipReplicatedCsrfFilter.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.security.csrf;
+
+import org.springframework.security.web.csrf.CsrfFilter;
+import org.springframework.security.web.util.matcher.AndRequestMatcher;
+import org.springframework.security.web.util.matcher.NegatedRequestMatcher;
+import org.springframework.security.web.util.matcher.RequestHeaderRequestMatcher;
+import org.springframework.security.web.util.matcher.RequestMatcher;
+import org.springframework.web.filter.OncePerRequestFilter;
+
+import javax.servlet.FilterChain;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+
+/**
+ * Skip Replicated Cross-Site Request Forgery Filter disables subsequent filtering for matched requests
+ */
+public class SkipReplicatedCsrfFilter extends OncePerRequestFilter {
+    /** RequestReplicator.REQUEST_TRANSACTION_ID_HEADER applied to replicated cluster requests */
+    protected static final String REPLICATED_REQUEST_HEADER = "X-RequestTransactionId";
+
+    /** Requests containing replicated header and not containing authorization cookies will be skipped */
+    private static final RequestMatcher REQUEST_MATCHER = new AndRequestMatcher(
+            new RequestHeaderRequestMatcher(REPLICATED_REQUEST_HEADER),
+            new NegatedRequestMatcher(new CsrfCookieRequestMatcher())
+    );
+
+    /**
+     * Set request attribute to disable standard CSRF Filter when request matches
+     *
+     * @param request HTTP Servlet Request
+     * @param response HTTP Servlet Response
+     * @param filterChain Filter Chain
+     * @throws ServletException Thrown on FilterChain.doFilter()
+     * @throws IOException Thrown on FilterChain.doFilter()
+     */
+    @Override
+    protected void doFilterInternal(final HttpServletRequest request, final HttpServletResponse response, final FilterChain filterChain) throws ServletException, IOException {
+        if (REQUEST_MATCHER.matches(request)) {
+            CsrfFilter.skipRequest(request);
+        }
+        filterChain.doFilter(request, response);
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/csrf/SkipReplicatedCsrfFilterTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/csrf/SkipReplicatedCsrfFilterTest.java
new file mode 100644
index 0000000000..a2e0727c44
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/csrf/SkipReplicatedCsrfFilterTest.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.security.csrf;
+
+import org.apache.nifi.web.security.http.SecurityCookieName;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+import org.springframework.http.HttpMethod;
+import org.springframework.mock.web.MockHttpServletRequest;
+import org.springframework.mock.web.MockHttpServletResponse;
+import org.springframework.security.web.csrf.CsrfFilter;
+
+import javax.servlet.FilterChain;
+import javax.servlet.ServletException;
+import javax.servlet.http.Cookie;
+
+import java.io.IOException;
+import java.util.UUID;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.verify;
+
+@ExtendWith(MockitoExtension.class)
+class SkipReplicatedCsrfFilterTest {
+    private static final String SHOULD_NOT_FILTER = String.format("SHOULD_NOT_FILTER%s", CsrfFilter.class.getName());
+
+    MockHttpServletRequest httpServletRequest;
+
+    MockHttpServletResponse httpServletResponse;
+
+    @Mock
+    FilterChain filterChain;
+
+    SkipReplicatedCsrfFilter filter;
+
+    @BeforeEach
+    void setHandler() {
+        filter = new SkipReplicatedCsrfFilter();
+        httpServletRequest = new MockHttpServletRequest();
+        httpServletResponse = new MockHttpServletResponse();
+    }
+
+    @Test
+    void testDoFilterInternalNotSkipped() throws ServletException, IOException {
+        httpServletRequest.setMethod(HttpMethod.GET.name());
+
+        filter.doFilterInternal(httpServletRequest, httpServletResponse, filterChain);
+
+        assertCsrfFilterNotSkipped();
+    }
+
+    @Test
+    void testDoFilterInternalBearerCookieNotSkipped() throws ServletException, IOException {
+        httpServletRequest.setMethod(HttpMethod.GET.name());
+        final Cookie bearerCookie = new Cookie(SecurityCookieName.AUTHORIZATION_BEARER.getName(), UUID.randomUUID().toString());
+        httpServletRequest.setCookies(bearerCookie);
+
+        filter.doFilterInternal(httpServletRequest, httpServletResponse, filterChain);
+
+        assertCsrfFilterNotSkipped();
+    }
+
+    @Test
+    void testDoFilterInternalReplicatedHeaderAndBearerCookieNotSkipped() throws ServletException, IOException {
+        httpServletRequest.setMethod(HttpMethod.GET.name());
+        httpServletRequest.addHeader(SkipReplicatedCsrfFilter.REPLICATED_REQUEST_HEADER, UUID.randomUUID().toString());
+        final Cookie bearerCookie = new Cookie(SecurityCookieName.AUTHORIZATION_BEARER.getName(), UUID.randomUUID().toString());
+        httpServletRequest.setCookies(bearerCookie);
+
+        filter.doFilterInternal(httpServletRequest, httpServletResponse, filterChain);
+
+        assertCsrfFilterNotSkipped();
+    }
+
+    @Test
+    void testDoFilterInternalReplicatedHeaderSkipped() throws ServletException, IOException {
+        httpServletRequest.setMethod(HttpMethod.GET.name());
+        httpServletRequest.addHeader(SkipReplicatedCsrfFilter.REPLICATED_REQUEST_HEADER, UUID.randomUUID().toString());
+
+        filter.doFilterInternal(httpServletRequest, httpServletResponse, filterChain);
+
+        final Object shouldNotFilter = httpServletRequest.getAttribute(SHOULD_NOT_FILTER);
+
+        assertEquals(Boolean.TRUE, shouldNotFilter);
+        verify(filterChain).doFilter(eq(httpServletRequest), eq(httpServletResponse));
+    }
+
+    private void assertCsrfFilterNotSkipped() throws ServletException, IOException {
+        final Object shouldNotFilter = httpServletRequest.getAttribute(SHOULD_NOT_FILTER);
+
+        assertNotEquals(Boolean.TRUE, shouldNotFilter);
+        verify(filterChain).doFilter(eq(httpServletRequest), eq(httpServletResponse));
+    }
+}


[nifi] 05/22: NIFI-5819: Support SQLServer sql_variant type

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit db2bf5f64530e66763894efcfa702d2b4bae1e39
Author: Matthew Burgess <ma...@apache.org>
AuthorDate: Mon Nov 21 18:35:35 2022 -0500

    NIFI-5819: Support SQLServer sql_variant type
    
    This closes #6699.
    
    Signed-off-by: Peter Turcsanyi <tu...@apache.org>
---
 .../src/main/java/org/apache/nifi/util/db/JdbcCommon.java         | 5 +++++
 .../org/apache/nifi/processors/standard/PutDatabaseRecord.java    | 8 ++++++++
 2 files changed, 13 insertions(+)

diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java b/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java
index b78408c912..ad7471a54c 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java
@@ -650,6 +650,11 @@ public class JdbcCommon {
                     builder.name(columnName).type().unionOf().nullBuilder().endNull().and().bytesType().endUnion().noDefault();
                     break;
 
+                case -150: // SQLServer may return -150 from the driver even though it's really -156 (sql_variant), treat as a union since we don't know what the values will actually be
+                case -156:
+                    builder.name(columnName).type().unionOf().nullBuilder().endNull().and().stringType().and().intType().and().longType().and().booleanType().and().bytesType().and()
+                            .doubleType().and().floatType().endUnion().noDefault();
+                    break;
 
                 default:
                     throw new IllegalArgumentException("createSchema: Unknown SQL type " + meta.getColumnType(i) + " / " + meta.getColumnTypeName(i)
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java
index 5e94200b44..3a1c8ca56e 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java
@@ -736,12 +736,20 @@ public class PutDatabaseRecord extends AbstractProcessor {
                             }
                         } else {
                             sqlType = column.getDataType();
+                            // SQLServer returns -150 for sql_variant from DatabaseMetaData though the server expects -156 when setting a sql_variant parameter
+                            if (sqlType == -150) {
+                                sqlType = -156;
+                            }
                         }
 
                         // Convert (if necessary) from field data type to column data type
                         if (fieldSqlType != sqlType) {
                             try {
                                 DataType targetDataType = DataTypeUtils.getDataTypeFromSQLTypeValue(sqlType);
+                                // If sqlType is unsupported, fall back to the fieldSqlType instead
+                                if (targetDataType == null) {
+                                    targetDataType = DataTypeUtils.getDataTypeFromSQLTypeValue(fieldSqlType);
+                                }
                                 if (targetDataType != null) {
                                     if (sqlType == Types.BLOB || sqlType == Types.BINARY) {
                                         if (currentValue instanceof Object[]) {


[nifi] 08/22: NIFI-10872 Fix broken Java Mail links in ConsumeIMAP and ConsumePOP3 documentation

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit bfa59b204115196ce563ada99634a29f32a142ed
Author: Nandor Soma Abonyi <ab...@gmail.com>
AuthorDate: Mon Nov 28 11:42:08 2022 +0100

    NIFI-10872 Fix broken Java Mail links in ConsumeIMAP and ConsumePOP3 documentation
    
    This closes #6723
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 .../org.apache.nifi.processors.email.ConsumeIMAP/additionalDetails.html | 2 +-
 .../org.apache.nifi.processors.email.ConsumePOP3/additionalDetails.html | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/resources/docs/org.apache.nifi.processors.email.ConsumeIMAP/additionalDetails.html b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/resources/docs/org.apache.nifi.processors.email.ConsumeIMAP/additionalDetails.html
index 93f09f65ac..dd51b768ec 100644
--- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/resources/docs/org.apache.nifi.processors.email.ConsumeIMAP/additionalDetails.html
+++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/resources/docs/org.apache.nifi.processors.email.ConsumeIMAP/additionalDetails.html
@@ -52,7 +52,7 @@
 	Another useful property is <b>mail.debug</b> which allows Java Mail API to print protocol messages to the console helping you to both understand what's going on as well as debug issues.
 	</p>
 	<p>
-	For the full list of available Java Mail properties please refer to <a href="http://connector.sourceforge.net/doc-files/Properties.html">here</a>
+	For the full list of available Java Mail properties please refer to <a href="https://javaee.github.io/javamail/docs/api/com/sun/mail/imap/package-summary.html">here</a>
 	</p>
 </body>
 </html>
diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/resources/docs/org.apache.nifi.processors.email.ConsumePOP3/additionalDetails.html b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/resources/docs/org.apache.nifi.processors.email.ConsumePOP3/additionalDetails.html
index f16557ee7b..40de3ba0e0 100644
--- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/resources/docs/org.apache.nifi.processors.email.ConsumePOP3/additionalDetails.html
+++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/resources/docs/org.apache.nifi.processors.email.ConsumePOP3/additionalDetails.html
@@ -51,7 +51,7 @@
 	Another useful property is	<b>mail.debug</b> which allows Java Mail API to print protocol messages to the console helping you to both understand what's going on as well as debug issues.
 	</p>
 	<p>
-	For the full list of available Java Mail properties please refer to <a href="http://connector.sourceforge.net/doc-files/Properties.html">here</a>
+	For the full list of available Java Mail properties please refer to <a href="https://javaee.github.io/javamail/docs/api/com/sun/mail/pop3/package-summary.html">here</a>
 	</p>
 </body>
 </html>


[nifi] 13/22: NIFI-10834 Updated TestPutSplunkHTTP to avoid non-deterministic 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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 03933c64253536e9422444f6640233a67147f634
Author: priyanka-28 <52...@users.noreply.github.com>
AuthorDate: Wed Nov 16 16:13:31 2022 -0600

    NIFI-10834 Updated TestPutSplunkHTTP to avoid non-deterministic tests
    
    This closes #6672
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 .../java/org/apache/nifi/processors/splunk/TestPutSplunkHTTP.java    | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git a/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/test/java/org/apache/nifi/processors/splunk/TestPutSplunkHTTP.java b/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/test/java/org/apache/nifi/processors/splunk/TestPutSplunkHTTP.java
index a9d39602b7..dd92cdf95a 100644
--- a/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/test/java/org/apache/nifi/processors/splunk/TestPutSplunkHTTP.java
+++ b/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/test/java/org/apache/nifi/processors/splunk/TestPutSplunkHTTP.java
@@ -44,6 +44,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
@@ -131,7 +132,7 @@ public class TestPutSplunkHTTP {
 
         // then
         testRunner.assertAllFlowFilesTransferred(PutSplunkHTTP.RELATIONSHIP_SUCCESS, 1);
-        assertEquals("/services/collector/raw?sourcetype=test%3Fsource%3Ftype&source=test_source", path.getValue());
+        assertTrue(path.getValue().startsWith("/services/collector/raw"));
     }
 
     @Test
@@ -163,7 +164,7 @@ public class TestPutSplunkHTTP {
 
         // then
         testRunner.assertAllFlowFilesTransferred(PutSplunkHTTP.RELATIONSHIP_SUCCESS, 1);
-        assertEquals("/services/collector/raw?host=test_host&index=test_index&sourcetype=test%3Fsource%3Ftype&source=test_source", path.getValue());
+        assertTrue(path.getValue().startsWith("/services/collector/raw"));
 
         assertEquals(EVENT, processor.getLastContent());
         assertEquals(attributes.get("ff_content_type"), processor.getLastContentType());


[nifi] 09/22: NIFI-10886 Upgraded PostgreSQL JDBC Driver from 42.4.1 to 42.4.3

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 83d9f9916a687e143d9157a10fd8a4b749ed71d2
Author: dependabot[bot] <49...@users.noreply.github.com>
AuthorDate: Wed Nov 23 22:22:46 2022 +0000

    NIFI-10886 Upgraded PostgreSQL JDBC Driver from 42.4.1 to 42.4.3
    
    This closes #6713
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 nifi-registry/nifi-registry-core/nifi-registry-test/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/nifi-registry/nifi-registry-core/nifi-registry-test/pom.xml b/nifi-registry/nifi-registry-core/nifi-registry-test/pom.xml
index e96e04ba92..72836f46ec 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-test/pom.xml
+++ b/nifi-registry/nifi-registry-core/nifi-registry-test/pom.xml
@@ -68,7 +68,7 @@
         <dependency>
             <groupId>org.postgresql</groupId>
             <artifactId>postgresql</artifactId>
-            <version>42.4.1</version>
+            <version>42.4.3</version>
         </dependency>
         <dependency>
             <groupId>junit</groupId>


[nifi] 17/22: NIFI-10890 Replaced HashMap with LinkedHashMap in JsonRowRecordReader

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit e6dfea6352f5ffabbbcafde10745c4fc59a64fc1
Author: sopan98 <ph...@gmail.com>
AuthorDate: Mon Nov 28 21:35:44 2022 -0600

    NIFI-10890 Replaced HashMap with LinkedHashMap in JsonRowRecordReader
    
    - Updated unit test based on deterministic behavior of LinkedHashMap
    
    This closes #6726
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 .../java/org/apache/nifi/json/AbstractJsonRowRecordReader.java    | 8 ++++----
 .../java/org/apache/nifi/json/TestJsonTreeRowRecordReader.java    | 2 +-
 2 files changed, 5 insertions(+), 5 deletions(-)

diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-json-record-utils/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-json-record-utils/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java
index b024beed6c..59f12770e9 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-json-record-utils/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-json-record-utils/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java
@@ -43,8 +43,8 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.text.DateFormat;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Iterator;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.function.BiPredicate;
@@ -119,7 +119,7 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
 
         this.strategy = strategy;
         this.captureFieldPredicate = captureFieldPredicate;
-        capturedFields = new HashMap<>();
+        capturedFields = new LinkedHashMap<>();
 
         try {
             jsonParser = jsonFactory.createParser(in);
@@ -306,7 +306,7 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
         final MapDataType mapDataType = (MapDataType) dataType;
         final DataType valueType = mapDataType.getValueType();
 
-        final Map<String, Object> mapValue = new HashMap<>();
+        final Map<String, Object> mapValue = new LinkedHashMap<>();
 
         final Iterator<Map.Entry<String, JsonNode>> fieldItr = fieldNode.fields();
         while (fieldItr.hasNext()) {
@@ -372,7 +372,7 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
 
     private Record createRecordFromRawValue(final JsonNode fieldNode, final RecordSchema childSchema) throws IOException {
         final Iterator<String> fieldNames = fieldNode.fieldNames();
-        final Map<String, Object> childValues = new HashMap<>();
+        final Map<String, Object> childValues = new LinkedHashMap<>();
         while (fieldNames.hasNext()) {
             final String childFieldName = fieldNames.next();
 
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonTreeRowRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonTreeRowRecordReader.java
index e8b0b6c48c..1f77aa6318 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonTreeRowRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonTreeRowRecordReader.java
@@ -423,7 +423,7 @@ class TestJsonTreeRowRecordReader {
         fields.add(new RecordField("id", RecordFieldType.INT.getDataType()));
         final RecordSchema schema = new SimpleRecordSchema(fields);
 
-        final String expectedMap = "{id=1, name=John Doe, address=123 My Street, city=My City, state=MS, zipCode=11111, country=USA, account=MapRecord[{balance=4750.89, id=42}]}";
+        final String expectedMap = "{id=1, name=John Doe, address=123 My Street, city=My City, state=MS, zipCode=11111, country=USA, account=MapRecord[{id=42, balance=4750.89}]}";
         final String expectedRecord = String.format("MapRecord[%s]", expectedMap);
         try (final InputStream in = new FileInputStream("src/test/resources/json/single-element-nested.json");
              final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, mock(ComponentLog.class), schema, dateFormat, timeFormat, timestampFormat)) {


[nifi] 14/22: NIFI-10859 Upgraded metrics-jvm from 4.1.2 to 4.1.33

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit ec813fc445ffc2d621287f54c99654c59b1f5dd5
Author: mr1716 <ms...@rit.edu>
AuthorDate: Sat Nov 26 07:57:59 2022 -0500

    NIFI-10859 Upgraded metrics-jvm from 4.1.2 to 4.1.33
    
    This closes #6721
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 nifi-commons/nifi-metrics/pom.xml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/nifi-commons/nifi-metrics/pom.xml b/nifi-commons/nifi-metrics/pom.xml
index 522b2f6c16..8903ab107b 100644
--- a/nifi-commons/nifi-metrics/pom.xml
+++ b/nifi-commons/nifi-metrics/pom.xml
@@ -68,7 +68,7 @@
         <dependency>
             <groupId>io.dropwizard.metrics</groupId>
             <artifactId>metrics-jvm</artifactId>
-            <version>4.1.2</version>
+            <version>4.1.33</version>
         </dependency>
         <dependency>
             <groupId>org.spockframework</groupId>
@@ -76,4 +76,4 @@
             <scope>test</scope>
         </dependency>
     </dependencies>
-</project>
\ No newline at end of file
+</project>


[nifi] 19/22: NIFI-10765 Added better error reporting in JASN1Reader

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit c36c4c0bbce4d3de690b58611a979373ce3d6ff0
Author: Tamas Palfy <tp...@apache.org>
AuthorDate: Tue Nov 22 19:24:55 2022 +0100

    NIFI-10765 Added better error reporting in JASN1Reader
    
    This closes #6703
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 .../nifi-asn1-bundle/nifi-asn1-services/pom.xml    |  34 ++---
 .../asn1bean/compiler/BerClassWriterFactory.java   |  37 +++++
 .../java/org/apache/nifi/jasn1/JASN1Reader.java    |  97 +++++++++++--
 .../additionalDetails.html                         |  66 ++++++++-
 .../org/apache/nifi/jasn1/JASN1ReaderTest.java     | 152 ++++++++++++++++++++-
 .../src/test/resources/cant_compile.asn            |  12 ++
 .../test/resources/cant_compile_mac_windows.asn    |  11 ++
 .../src/test/resources/cant_parse.asn              |  21 +++
 8 files changed, 402 insertions(+), 28 deletions(-)

diff --git a/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/pom.xml b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/pom.xml
index 69864bc88e..aab23e9363 100644
--- a/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/pom.xml
+++ b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/pom.xml
@@ -62,6 +62,12 @@
             <artifactId>caffeine</artifactId>
             <version>2.8.1</version>
         </dependency>
+        <dependency>
+            <groupId>antlr</groupId>
+            <artifactId>antlr</artifactId>
+            <version>2.7.7</version>
+        </dependency>
+
 
         <!-- For Test -->
         <dependency>
@@ -130,6 +136,9 @@
                         <exclude>src/test/resources/simple_types.asn</exclude>
                         <exclude>src/test/resources/complex_types.asn</exclude>
                         <exclude>src/test/resources/tbcd_string.asn</exclude>
+                        <exclude>src/test/resources/cant_parse.asn</exclude>
+                        <exclude>src/test/resources/cant_compile.asn</exclude>
+                        <exclude>src/test/resources/cant_compile_mac_windows.asn</exclude>
                         <exclude>src/test/resources/examples/basic-types.dat</exclude>
                         <exclude>src/test/resources/examples/composite.dat</exclude>
                         <exclude>src/test/resources/examples/tbcd-string.dat</exclude>
@@ -139,23 +148,14 @@
             <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-checkstyle-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <id>check-style</id>
-                        <phase>verify</phase>
-                        <configuration>
-                            <encoding>UTF-8</encoding>
-                            <suppressionsLocation>${basedir}/checkstyle-suppressions.xml</suppressionsLocation>
-                            <sourceDirectories>
-                                <sourceDirectory>${project.build.sourceDirectory}</sourceDirectory>
-                                <sourceDirectory>${project.build.testSourceDirectory}</sourceDirectory>
-                            </sourceDirectories>
-                        </configuration>
-                        <goals>
-                            <goal>check</goal>
-                        </goals>
-                    </execution>
-                </executions>
+                <configuration>
+                    <encoding>UTF-8</encoding>
+                    <suppressionsLocation>${basedir}/checkstyle-suppressions.xml</suppressionsLocation>
+                    <sourceDirectories>
+                        <sourceDirectory>${project.build.sourceDirectory}</sourceDirectory>
+                        <sourceDirectory>${project.build.testSourceDirectory}</sourceDirectory>
+                    </sourceDirectories>
+                </configuration>
             </plugin>
         </plugins>
     </build>
diff --git a/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/com/beanit/asn1bean/compiler/BerClassWriterFactory.java b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/com/beanit/asn1bean/compiler/BerClassWriterFactory.java
new file mode 100644
index 0000000000..857fef9794
--- /dev/null
+++ b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/com/beanit/asn1bean/compiler/BerClassWriterFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.beanit.asn1bean.compiler;
+
+import com.beanit.asn1bean.compiler.model.AsnModule;
+
+import java.nio.file.Path;
+import java.util.HashMap;
+
+public class BerClassWriterFactory {
+    public static BerClassWriter createBerClassWriter(HashMap<String, AsnModule> modulesByName, Path asnOutDir) {
+        BerClassWriter berClassWriter = new BerClassWriter(
+                modulesByName,
+                asnOutDir.toString(),
+                "",
+                true,
+                false,
+                false
+        );
+
+        return berClassWriter;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/JASN1Reader.java b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/JASN1Reader.java
index 873cdf042f..c2bbd99761 100644
--- a/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/JASN1Reader.java
+++ b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/JASN1Reader.java
@@ -16,6 +16,14 @@
  */
 package org.apache.nifi.jasn1;
 
+import antlr.RecognitionException;
+import antlr.TokenStreamException;
+import com.beanit.asn1bean.compiler.BerClassWriter;
+import com.beanit.asn1bean.compiler.BerClassWriterFactory;
+import com.beanit.asn1bean.compiler.model.AsnModel;
+import com.beanit.asn1bean.compiler.model.AsnModule;
+import com.beanit.asn1bean.compiler.parser.ASNLexer;
+import com.beanit.asn1bean.compiler.parser.ASNParser;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnDisabled;
@@ -38,11 +46,14 @@ import org.apache.nifi.serialization.RecordReader;
 import org.apache.nifi.serialization.RecordReaderFactory;
 import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
 
+import javax.tools.DiagnosticCollector;
 import javax.tools.JavaCompiler;
 import javax.tools.JavaFileObject;
 import javax.tools.StandardJavaFileManager;
 import javax.tools.ToolProvider;
 import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.nio.file.Files;
@@ -51,8 +62,13 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.Collectors;
 
 @Tags({"asn", "ans1", "jasn.1", "jasn1", "record", "reader", "parser"})
@@ -125,7 +141,7 @@ public class JASN1Reader extends AbstractConfigurableComponent implements Record
     );
 
     private String identifier;
-    private ComponentLog logger;
+    ComponentLog logger;
 
     private RecordSchemaProvider schemaProvider = new RecordSchemaProvider();
 
@@ -219,10 +235,38 @@ public class JASN1Reader extends AbstractConfigurableComponent implements Record
         asnCompilerArguments.add("-o");
         asnCompilerArguments.add(asnOutDir.toString());
 
+        HashMap<String, AsnModule> modulesByName = new HashMap<>();
+
+        Exception parseException = null;
+        for (String asn1File : asnFilePaths) {
+            logger.info("Parsing " + asn1File);
+            try {
+                AsnModel model = getJavaModelFromAsn1File(asn1File);
+                modulesByName.putAll(model.modulesByName);
+            } catch (FileNotFoundException e) {
+                logger.error("ASN.1 file not found [{}]", asn1File, e);
+                parseException = e;
+            } catch (TokenStreamException | RecognitionException e) {
+                logger.error("ASN.1 stream parsing failed [{}]", asn1File, e);
+                parseException = e;
+            } catch (Exception e) {
+                logger.error("ASN.1 parsing failed [{}]", asn1File, e);
+                parseException = e;
+            }
+        }
+
+        if (parseException != null) {
+            throw new ProcessException("ASN.1 parsing failed", parseException);
+        }
+
         try {
-            com.beanit.asn1bean.compiler.Compiler.main(asnCompilerArguments.toArray(new String[0]));
+            logger.info("Writing ASN.1 classes to directory [{}]", asnOutDir);
+
+            BerClassWriter classWriter = BerClassWriterFactory.createBerClassWriter(modulesByName, asnOutDir);
+
+            classWriter.translate();
         } catch (Exception e) {
-            throw new ProcessException("Couldn't compile asn files to java.", e);
+            throw new ProcessException("ASN.1 compilation failed", e);
         }
 
         List<File> javaFiles;
@@ -234,7 +278,7 @@ public class JASN1Reader extends AbstractConfigurableComponent implements Record
                 .map(File::new)
                 .collect(Collectors.toList());
         } catch (IOException e) {
-            throw new ProcessException("Couldn't access '" + asnOutDir + "'");
+            throw new ProcessException("Access directory failed " + asnOutDir);
         }
 
         JavaCompiler javaCompiler = ToolProvider.getSystemJavaCompiler();
@@ -246,10 +290,17 @@ public class JASN1Reader extends AbstractConfigurableComponent implements Record
         Iterable<? extends JavaFileObject> units;
         units = fileManager.getJavaFileObjectsFromFiles(javaFiles);
 
-        JavaCompiler.CompilationTask task = javaCompiler.getTask(null, fileManager, null, optionList, null, units);
+        DiagnosticCollector<JavaFileObject> diagnosticListener = new DiagnosticCollector<>();
+        JavaCompiler.CompilationTask task = javaCompiler.getTask(null, fileManager, diagnosticListener, optionList, null, units);
+
         Boolean success = task.call();
         if (!success) {
-            throw new ProcessException("Couldn't compile java file.");
+            Set<String> errorMessages = new LinkedHashSet();
+            diagnosticListener.getDiagnostics().stream().map(d -> d.getMessage(Locale.getDefault())).forEach(errorMessages::add);
+
+            errorMessages.forEach(logger::error);
+
+            throw new ProcessException("Java compilation failed");
         }
     }
 
@@ -266,7 +317,7 @@ public class JASN1Reader extends AbstractConfigurableComponent implements Record
                     .map(Path::toFile)
                     .forEach(File::delete);
             } catch (IOException e) {
-                throw new ProcessException("Couldn't delete '" + asnOutDir + "'");
+                throw new ProcessException("Delete directory failed " + asnOutDir);
             }
         }
     }
@@ -294,6 +345,37 @@ public class JASN1Reader extends AbstractConfigurableComponent implements Record
         return new JASN1RecordReader(rootClassName, recordField, schemaProvider, customClassLoader, iteratorProviderClassName, in, logger);
     }
 
+    AsnModel getJavaModelFromAsn1File(String inputFileName)
+            throws FileNotFoundException, TokenStreamException, RecognitionException {
+
+        InputStream stream = new FileInputStream(inputFileName);
+        ASNLexer lexer = new ASNLexer(stream);
+
+        AtomicBoolean parseError = new AtomicBoolean(false);
+        ASNParser parser = new ASNParser(lexer) {
+            @Override
+            public void reportError(String s) {
+                logger.error("{} - {}", inputFileName, s);
+                parseError.set(true);
+            }
+
+            @Override
+            public void reportError(RecognitionException e) {
+                logger.error("{} - {}", inputFileName, e.toString());
+                parseError.set(true);
+            }
+        };
+
+        if (parseError.get()) {
+            throw new ProcessException("ASN.1 parsing failed");
+        }
+
+        AsnModel model = new AsnModel();
+        parser.module_definitions(model);
+
+        return model;
+    }
+
     String guessRootClassName(String rootModelName) {
         try {
             StringBuilder rootClassNameBuilder = new StringBuilder();
@@ -311,5 +393,4 @@ public class JASN1Reader extends AbstractConfigurableComponent implements Record
             throw new ProcessException("Couldn't infer root model name from '" + rootModelName + "'", e);
         }
     }
-
 }
diff --git a/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/resources/docs/org.apache.nifi.jasn1.JASN1Reader/additionalDetails.html b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/resources/docs/org.apache.nifi.jasn1.JASN1Reader/additionalDetails.html
index 4af933d456..31edf6d64e 100644
--- a/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/resources/docs/org.apache.nifi.jasn1.JASN1Reader/additionalDetails.html
+++ b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/resources/docs/org.apache.nifi.jasn1.JASN1Reader/additionalDetails.html
@@ -27,13 +27,19 @@
             This service creates record readers for ASN.1 input.
         </p>
         <p>
-            ASN.1 schema files (with full path) can be defined via the <i>Root Model Name</i> property as a comma separated list.
+            ASN.1 schema files (with full path) can be defined via the <i>ASN.1 Files</i> property as a comma separated list.
             The controller service preprocesses these files and generates sources that it uses for parsing data later.<br/>
             <b>
                 Note that this preprocessing may take a while, especially when the schema files are large.
                 The service remains in the <i>Enabling</i> state until this preprocessing is finished.
                 Processors using the service are ready to be started at this point but probably won't work properly until the service is fully <i>Enabled</i>.
             </b>
+            <br />
+            <b>
+                Also note that the preprocessing phase can fail if there are problems with the ASN.1 schema files. The bulletin - as per usual -
+                will show error messages related to the failure but interpreting those messages may not be straightforward.
+                For help troubleshooting such messages please refer to the <i>Troubleshooting</i> section below.
+            </b>
         </p>
         <p>
             The root model type can be defined via the <i>Root Model Name</i> property. It's format should be "MODULE-NAME.ModelType".
@@ -58,5 +64,63 @@
             To be able to set the property the service needs to be disabled in the end - and let it remove the directory,
                 however this shouldn't be an issue as the name of the root model class will be the same in the new temporary directory.)
         </p>
+
+        <h3>Troubleshooting</h3>
+
+        <p>
+            The preprocessing is done in two phases:
+            <ol>
+                <li>
+                    The first phase reads the ASN.1 schema files and parses them. Formatting errors are usually reported during this phase.
+                    Here are some possible error messages and the potential root causes of the issues:
+                    <ul>
+                        <li>
+                            <i>line NNN:MMM: unexpected token: someFieldName</i> - On the NNNth line, starting at the MMMth position, <i>someFieldName</i> is encountered
+                            which was unexpected. Usually this means <i>someFieldName</i> itself is fine but the previous field declaration doesn't have a comma ',' at the end.
+                        </li>
+                        <li>
+                            <i>line NNN:MMM: unexpected token: [</i> - On the NNNth line, starting at the MMMth position, the opening square bracket '[' is encountered
+                            which was unexpected. Usually this is the index part of the field declaration and for some reason the field declaration is invalid. This can typically
+                            occur if the field name is invalid, e.g. starts with an uppercase letter. (Field names must start with a lowercase letter.)
+                        </li>
+                    </ul>
+                </li>
+                <li>
+                    The second phase compiles the ASN.1 schema files into Java classes. Even if the ASN.1 files meet the formal requirements, due to the nature of the created Java
+                    files there are some extra limitations:
+                    <ul>
+                        <li>
+                            On certain systems type names are treated as case-insensitive. Because of this, two types whose names only differ in the cases of their letters may cause errors.
+                            For example if the ASN.1 schema files define both 'SameNameWithDifferentCase' and 'SAMENAMEWithDifferentCase', the following error may be reported:
+                            <br />
+                            <br />
+                            <i>
+                                class SAMENAMEWithDifferentCase is public, should be declared in a file named SAMENAMEWithDifferentCase.java
+                            </i>
+                        </li>
+                        <li>
+                            Certain keywords cannot be used as field names. Known reserved keywords and the corresponding reported error messages are:
+                        </li>
+                        <ul>
+                            <li>
+                                length
+                            </li>
+                            <br />
+                            <i>
+                                incompatible types: com.beanit.asn1bean.ber.types.BerInteger cannot be converted to com.beanit.asn1bean.ber.BerLength
+                            </i>
+                            <br />
+                            <i>
+                                incompatible types: boolean cannot be converted to java.io.OutputStream
+                            </i>
+                            <br />
+                            <i>
+                                Some messages have been simplified; recompile with -Xdiags:verbose to get full output
+                            </i>
+                        </ul>
+                    </ul>
+                </li>
+            </ol>
+        </p>
     </body>
 </html>
diff --git a/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/java/org/apache/nifi/jasn1/JASN1ReaderTest.java b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/java/org/apache/nifi/jasn1/JASN1ReaderTest.java
index 987d01e592..98377de503 100644
--- a/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/java/org/apache/nifi/jasn1/JASN1ReaderTest.java
+++ b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/java/org/apache/nifi/jasn1/JASN1ReaderTest.java
@@ -19,18 +19,35 @@ package org.apache.nifi.jasn1;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ControllerServiceInitializationContext;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.exception.ProcessException;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.condition.EnabledOnOs;
+import org.junit.jupiter.api.condition.OS;
+import org.mockito.ArgumentCaptor;
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
 
+import java.io.FileNotFoundException;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.List;
+import java.util.StringJoiner;
+
 import static org.apache.nifi.jasn1.JASN1Reader.ASN_FILES;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Answers.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 public class JASN1ReaderTest {
@@ -66,7 +83,7 @@ public class JASN1ReaderTest {
         // GIVEN
         ConfigurationContext context = mock(ConfigurationContext.class, RETURNS_DEEP_STUBS);
         when(context.getProperty(ASN_FILES).isSet()).thenReturn(true);
-        when(context.getProperty(ASN_FILES).evaluateAttributeExpressions().getValue()).thenReturn("src/test/resources/test.asn");
+        when(context.getProperty(ASN_FILES).evaluateAttributeExpressions().getValue()).thenReturn(Paths.get("src", "test", "resources", "test.asn").toString());
 
         // WHEN
         testSubject.onEnabled(context);
@@ -78,4 +95,135 @@ public class JASN1ReaderTest {
         assertEquals("org.apache.nifi.jasn1.test.RootType", actualRootModelName);
         assertNotNull(actual);
     }
+
+    @Test
+    public void testAsnFileDoesntExist() throws Exception {
+        // GIVEN
+        ConfigurationContext context = mock(ConfigurationContext.class, RETURNS_DEEP_STUBS);
+        when(context.getProperty(ASN_FILES).isSet()).thenReturn(true);
+        when(context.getProperty(ASN_FILES).evaluateAttributeExpressions().getValue()).thenReturn(
+                new StringJoiner(",")
+                        .add(Paths.get("src", "test", "resources", "test.asn").toString())
+                        .add(Paths.get("src", "test", "resources", "doesnt_exist.asn").toString())
+                        .toString()
+        );
+
+        // WHEN
+        ProcessException processException = assertThrows(
+                ProcessException.class,
+                () -> testSubject.onEnabled(context)
+        );
+        Throwable cause = processException.getCause();
+
+        assertEquals(FileNotFoundException.class, cause.getClass());
+        assertThat(cause.getMessage(), containsString("doesnt_exist.asn"));
+    }
+
+    @Test
+    /*
+     * Checks reported messages of underlying libraries that are explained in additionalDetails.html.
+     * In case of changes to this test additionalDetails.html may need to be updated as well.
+     */
+    public void testCantParseAsn() throws Exception {
+        // GIVEN
+        String asnFile = Paths.get("src", "test", "resources", "cant_parse.asn").toString();
+
+        List<String> expectedErrorMessages = Arrays.asList(
+                "line 11:5: unexpected token: field3",
+                "line 17:33: unexpected token: ["
+        );
+
+        // WHEN
+        // THEN
+        testParseError(asnFile, expectedErrorMessages);
+    }
+
+    @Test
+    /*
+     * Checks reported messages of underlying libraries that are explained in additionalDetails.html.
+     * In case of changes to this test additionalDetails.html may need to be updated as well.
+     */
+    public void testCantCompileAsn() throws Exception {
+        // GIVEN
+        String asnFiles = Paths.get("src", "test", "resources", "cant_compile.asn").toString();
+
+        List<String> expectedErrorMessages = Arrays.asList(
+                ".*com\\.beanit\\.asn1bean\\.ber\\.types\\.BerInteger.*com\\.beanit\\.asn1bean\\.ber\\.BerLength.*",
+                ".*boolean.*java\\.io\\.OutputStream.*",
+                ".*-Xdiags:verbose.*"
+        );
+
+        // WHEN
+        // THEN
+        testCompileError(asnFiles, expectedErrorMessages);
+    }
+
+    @EnabledOnOs({ OS.MAC, OS.WINDOWS })
+    @Test
+    /*
+     * Checks reported messages of underlying libraries that are explained in additionalDetails.html.
+     * In case of changes to this test additionalDetails.html may need to be updated as well.
+     */
+    public void testCantCompileAsnOnMacWindows() throws Exception {
+        // GIVEN
+        String asnFiles = Paths.get("src", "test", "resources", "cant_compile_mac_windows.asn").toString();
+
+        List<String> expectedErrorMessages = Arrays.asList(
+                ".*SAMENAMEWithDifferentCase.*SAMENAMEWithDifferentCase.*"
+        );
+
+        // WHEN
+        // THEN
+        testCompileError(asnFiles, expectedErrorMessages);
+    }
+
+    private void testParseError(String asnFile, List<String> expectedErrorMessages) {
+        // GIVEN
+        ConfigurationContext context = mock(ConfigurationContext.class, RETURNS_DEEP_STUBS);
+        when(context.getProperty(ASN_FILES).isSet()).thenReturn(true);
+        when(context.getProperty(ASN_FILES).evaluateAttributeExpressions().getValue())
+                .thenReturn(asnFile);
+
+
+        // WHEN
+        assertThrows(
+                ProcessException.class,
+                () -> testSubject.onEnabled(context)
+        );
+
+        // THEN
+        ArgumentCaptor<String> errorCaptor = ArgumentCaptor.forClass(String.class);
+        verify(testSubject.logger, atLeastOnce()).error(eq("{} - {}"), anyString(), errorCaptor.capture());
+
+        List<String> actualErrorMessages = errorCaptor.getAllValues();
+
+        assertEquals(expectedErrorMessages, actualErrorMessages);
+    }
+
+    private void testCompileError(String asnFiles, List<String> expectedErrorMessages) {
+        // GIVEN
+        ConfigurationContext context = mock(ConfigurationContext.class, RETURNS_DEEP_STUBS);
+        when(context.getProperty(ASN_FILES).isSet()).thenReturn(true);
+        when(context.getProperty(ASN_FILES).evaluateAttributeExpressions().getValue())
+                .thenReturn(asnFiles);
+
+        // WHEN
+        assertThrows(
+                ProcessException.class,
+                () -> testSubject.onEnabled(context)
+        );
+
+        // THEN
+        ArgumentCaptor<String> errorCaptor = ArgumentCaptor.forClass(String.class);
+        verify(testSubject.logger, atLeastOnce()).error(errorCaptor.capture());
+
+        List<String> actualErrorMessages = errorCaptor.getAllValues();
+        assertEquals(expectedErrorMessages.size(), actualErrorMessages.size());
+
+        for (int errorMessageIndex = 0; errorMessageIndex < actualErrorMessages.size(); errorMessageIndex++) {
+            String expectedErrorMessage = expectedErrorMessages.get(errorMessageIndex);
+            String actualErrorMessage = actualErrorMessages.get(errorMessageIndex);
+            assertTrue(actualErrorMessage.matches(expectedErrorMessage), "Expected string matching '" + expectedErrorMessage + "', got '" + actualErrorMessage + "'");
+        }
+    }
 }
diff --git a/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/resources/cant_compile.asn b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/resources/cant_compile.asn
new file mode 100644
index 0000000000..bf9bc62b39
--- /dev/null
+++ b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/resources/cant_compile.asn
@@ -0,0 +1,12 @@
+ORG-APACHE-NIFI-JASN1-TEST
+
+DEFINITIONS IMPLICIT TAGS ::=
+
+BEGIN
+
+TypeWithReservedJavaKeyword ::= SEQUENCE
+{
+    length                      [0] INTEGER
+}
+
+END
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/resources/cant_compile_mac_windows.asn b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/resources/cant_compile_mac_windows.asn
new file mode 100644
index 0000000000..688347735c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/resources/cant_compile_mac_windows.asn
@@ -0,0 +1,11 @@
+ORG-APACHE-NIFI-JASN1-TEST
+
+DEFINITIONS IMPLICIT TAGS ::=
+
+BEGIN
+
+SameNameWithDifferentCase ::= OCTET STRING
+
+SAMENAMEWithDifferentCase ::= OCTET STRING
+
+END
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/resources/cant_parse.asn b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/resources/cant_parse.asn
new file mode 100644
index 0000000000..c0131f34ac
--- /dev/null
+++ b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/resources/cant_parse.asn
@@ -0,0 +1,21 @@
+ORG-APACHE-NIFI-JASN1-TEST
+
+DEFINITIONS IMPLICIT TAGS ::=
+
+BEGIN
+
+MissingComma ::= SEQUENCE
+{
+    field1                      [0] BOOLEAN,
+    field2MissingComma          [1] BOOLEAN
+    field3                      [2] BOOLEAN
+}
+
+Uppercase ::= SEQUENCE
+{
+    field1                      [0] BOOLEAN,
+    Field2Uppercase             [1] BOOLEAN,
+    field3                      [2] BOOLEAN
+}
+
+END
\ No newline at end of file


[nifi] 02/22: NIFI-10850: Fixed Query Parameters property in InvokeAWSGatewayApi should support FlowFile attributes in EL

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 2834aab432ed1c43627a54e8445fba3ab00de366
Author: Peter Turcsanyi <tu...@apache.org>
AuthorDate: Mon Nov 21 17:27:31 2022 +0100

    NIFI-10850: Fixed Query Parameters property in InvokeAWSGatewayApi should support FlowFile attributes in EL
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #6692.
---
 .../nifi/processors/aws/wag/AbstractAWSGatewayApiProcessor.java  | 9 +++++----
 .../nifi/processors/aws/wag/TestInvokeAmazonGatewayApiMock.java  | 5 +++--
 2 files changed, 8 insertions(+), 6 deletions(-)

diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/wag/AbstractAWSGatewayApiProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/wag/AbstractAWSGatewayApiProcessor.java
index 7cde727fa0..66068a863e 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/wag/AbstractAWSGatewayApiProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/wag/AbstractAWSGatewayApiProcessor.java
@@ -402,7 +402,7 @@ public abstract class AbstractAWSGatewayApiProcessor extends
                 context.getProperty(PROP_METHOD).evaluateAttributeExpressions(requestFlowFile)
                         .getValue()).toUpperCase();
         final HttpMethodName methodName = HttpMethodName.fromValue(method);
-        return configureRequest(context, session, resourcePath,requestFlowFile, methodName, attributes);
+        return configureRequest(context, session, resourcePath, requestFlowFile, methodName, attributes);
     }
 
     protected GenericApiGatewayRequest configureRequest(final ProcessContext context,
@@ -414,7 +414,7 @@ public abstract class AbstractAWSGatewayApiProcessor extends
 
         GenericApiGatewayRequestBuilder builder = new GenericApiGatewayRequestBuilder()
             .withResourcePath(resourcePath);
-        final Map<String, List<String>> parameters = getParameters(context);
+        final Map<String, List<String>> parameters = getParameters(context, attributes);
         builder = builder.withParameters(parameters);
 
         InputStream requestBody;
@@ -522,15 +522,16 @@ public abstract class AbstractAWSGatewayApiProcessor extends
      * Returns a map of Query Parameter Name to Values
      *
      * @param context ProcessContext
+     * @param flowFileAttributes map of FlowFile attributes used for EL evaluation
      * @return Map of names and values
      */
-    protected Map<String, List<String>> getParameters(final ProcessContext context) {
+    protected Map<String, List<String>> getParameters(final ProcessContext context, Map<String, String> flowFileAttributes) {
 
         if (!context.getProperty(PROP_QUERY_PARAMS).isSet()) {
             return new HashMap<>();
         }
         final String queryString = context.getProperty(PROP_QUERY_PARAMS)
-                                          .evaluateAttributeExpressions().getValue();
+                                          .evaluateAttributeExpressions(flowFileAttributes).getValue();
         final List<NameValuePair> params = URLEncodedUtils
             .parse(queryString, Charsets.toCharset("UTF-8"));
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeAmazonGatewayApiMock.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeAmazonGatewayApiMock.java
index 5b7927dd31..b3a4a970e8 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeAmazonGatewayApiMock.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeAmazonGatewayApiMock.java
@@ -178,7 +178,7 @@ public class TestInvokeAmazonGatewayApiMock {
 
         // add dynamic property
         runner.setProperty("dynamicHeader", "yes!");
-        runner.setProperty(InvokeAWSGatewayApi.PROP_QUERY_PARAMS, "apples=oranges&dogs=cats");
+        runner.setProperty(InvokeAWSGatewayApi.PROP_QUERY_PARAMS, "apples=oranges&dogs=cats&filename=${filename}");
 
         // set the regex
         runner.setProperty(InvokeAWSGatewayApi.PROP_ATTRIBUTES_TO_SEND, "F.*");
@@ -186,6 +186,7 @@ public class TestInvokeAmazonGatewayApiMock {
         final Map<String, String> attributes = new HashMap<>();
         attributes.put(CoreAttributes.MIME_TYPE.key(), "application/plain-text");
         attributes.put("Foo", "Bar");
+        attributes.put("filename", "testfile");
         runner.enqueue("Hello".getBytes(StandardCharsets.UTF_8), attributes);
         // execute
         runner.assertValid();
@@ -197,7 +198,7 @@ public class TestInvokeAmazonGatewayApiMock {
                                 && argument.getFirstHeader("Authorization").getValue().startsWith("AWS4")
                                 && argument.getFirstHeader("dynamicHeader").getValue().equals("yes!")
                                 && argument.getFirstHeader("Foo").getValue().equals("Bar")
-                                && argument.getURI().toString().equals("https://foobar.execute-api.us-east-1.amazonaws.com/TEST?dogs=cats&apples=oranges")),
+                                && argument.getURI().toString().equals("https://foobar.execute-api.us-east-1.amazonaws.com/TEST?filename=testfile&dogs=cats&apples=oranges")),
                         any(HttpContext.class));
         // check
         runner.assertTransferCount(InvokeAWSGatewayApi.REL_SUCCESS_REQ, 1);


[nifi] 12/22: NIFI-10875 Changed TestQuerySolr to check for path elements

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 2e6f8ecbe00561b3a348666d82146004f7624b56
Author: Yihen Liu <tt...@gmail.com>
AuthorDate: Thu Nov 24 13:37:29 2022 -0600

    NIFI-10875 Changed TestQuerySolr to check for path elements
    
    - Changes avoid non-deterministic expectations for query parameters
    
    This closes #6718
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 .../apache/nifi/processors/solr/TestQuerySolr.java | 24 ++++++++++++++++------
 1 file changed, 18 insertions(+), 6 deletions(-)

diff --git a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/TestQuerySolr.java b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/TestQuerySolr.java
index 60cb32f117..0ea1db65da 100644
--- a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/TestQuerySolr.java
+++ b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/TestQuerySolr.java
@@ -602,8 +602,12 @@ public class TestQuerySolr {
 
         assertEquals(SOLR_CONNECT, attributes.get(QuerySolr.ATTRIBUTE_SOLR_CONNECT));
         assertEquals(DEFAULT_SOLR_CORE, attributes.get(QuerySolr.ATTRIBUTE_SOLR_COLLECTION));
-
-        assertEquals("q=*:*&qt=/select&start=0&rows=10&stats=true&facet=true", attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("q=*:*"));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("qt=/select"));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("start=0"));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("rows=10"));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("stats=true"));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("facet=true"));
         assertEquals("0", attributes.get(QuerySolr.ATTRIBUTE_SOLR_STATUS));
         assertEquals("0", attributes.get(QuerySolr.ATTRIBUTE_SOLR_START));
         assertEquals("10", attributes.get(QuerySolr.ATTRIBUTE_SOLR_ROWS));
@@ -626,8 +630,12 @@ public class TestQuerySolr {
 
         assertEquals(SOLR_CONNECT, attributes.get(QuerySolr.ATTRIBUTE_SOLR_CONNECT));
         assertEquals(DEFAULT_SOLR_CORE, attributes.get(QuerySolr.ATTRIBUTE_SOLR_COLLECTION));
-
-        assertEquals("q=*:*&qt=/select&start=0&rows=10&stats=true&facet=true", attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("q=*:*"));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("qt=/select"));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("start=0"));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("rows=10"));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("stats=true"));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("facet=true"));
         assertEquals("0", attributes.get(QuerySolr.ATTRIBUTE_SOLR_STATUS));
         assertEquals("0", attributes.get(QuerySolr.ATTRIBUTE_SOLR_START));
         assertEquals("10", attributes.get(QuerySolr.ATTRIBUTE_SOLR_ROWS));
@@ -650,8 +658,12 @@ public class TestQuerySolr {
 
         assertEquals(SOLR_CONNECT, attributes.get(QuerySolr.ATTRIBUTE_SOLR_CONNECT));
         assertEquals(DEFAULT_SOLR_CORE, attributes.get(QuerySolr.ATTRIBUTE_SOLR_COLLECTION));
-
-        assertEquals("q=*:*&qt=/select&start=0&rows=10&stats=true&facet=true", attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("q=*:*"));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("qt=/select"));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("start=0"));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("rows=10"));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("stats=true"));
+        assertTrue(attributes.get(QuerySolr.ATTRIBUTE_SOLR_QUERY).contains("facet=true"));
         assertEquals("0", attributes.get(QuerySolr.ATTRIBUTE_SOLR_STATUS));
         assertEquals("0", attributes.get(QuerySolr.ATTRIBUTE_SOLR_START));
         assertEquals("10", attributes.get(QuerySolr.ATTRIBUTE_SOLR_ROWS));


[nifi] 21/22: NIFI-10901 - PublishKafka headers not sent in ProducerRecord (#6731)

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 6808a5944d328305ba67babd3e86cfd5e783bcf5
Author: greyp9 <gr...@users.noreply.github.com>
AuthorDate: Wed Nov 30 13:23:23 2022 -0500

    NIFI-10901 - PublishKafka headers not sent in ProducerRecord (#6731)
---
 .../processors/kafka/pubsub/PublisherLease.java    |  3 +-
 .../additionalDetails.html                         |  2 +-
 .../pubsub/TestPublishKafkaMockParameterized.java  | 97 +++++++++-------------
 ...> TestPublishKafkaRecordMockParameterized.java} | 49 +++++++++--
 .../Publish/parameterized/flowfileInput1.json      |  8 ++
 .../Publish/parameterized/flowfileInputA.json      | 12 +++
 .../Publish/parameterized/kafkaOutput1A.json       | 18 ++++
 .../Publish/parameterized/kafkaOutput1B.json       | 18 ++++
 .../Publish/parameterized/kafkaOutputA1.json       | 22 +++++
 .../Publish/parameterized/kafkaOutputA2.json       | 22 +++++
 .../parameterized/flowfileInputDoc1V.json          |  8 ++
 .../parameterized/flowfileInputDoc1W.json          | 15 ++++
 .../parameterized/flowfileInputDoc2W.json          | 15 ++++
 .../parameterized/kafkaOutputDoc1V.json            | 21 +++++
 .../parameterized/kafkaOutputDoc1W.json            | 18 ++++
 .../parameterized/kafkaOutputDoc2W.json            | 16 ++++
 16 files changed, 275 insertions(+), 69 deletions(-)

diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java
index 0949e537b6..da37db0319 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java
@@ -431,7 +431,8 @@ public class PublisherLease implements Closeable {
     }
 
     protected void publish(final FlowFile flowFile, final byte[] messageKey, final byte[] messageContent, final String topic, final InFlightMessageTracker tracker, final Integer partition) {
-        publish(flowFile, Collections.emptyList(), messageKey, messageContent, topic, tracker, partition);
+        final List<Header> headers = toHeaders(flowFile, Collections.emptyMap());
+        publish(flowFile, headers, messageKey, messageContent, topic, tracker, partition);
     }
 
     protected void publish(final FlowFile flowFile, final List<Header> headers, final byte[] messageKey, final byte[] messageContent,
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafkaRecord_2_6/additionalDetails.html b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafkaRecord_2_6/additionalDetails.html
index ce87abc73d..e125b0f067 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafkaRecord_2_6/additionalDetails.html
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafkaRecord_2_6/additionalDetails.html
@@ -375,7 +375,7 @@
             <table border="thin">
                 <tr>
                     <th>Record Key</th>
-                    <td><code>Acme Accounts</code></td>
+                    <td><code>Acme Holdings</code></td>
                 </tr>
                 <tr>
                     <th>Record Value</th>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaMockParameterized.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaMockParameterized.java
index 4e97a6da84..b15f5d55a1 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaMockParameterized.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaMockParameterized.java
@@ -31,17 +31,12 @@ import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.clients.producer.RecordMetadata;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.header.Header;
 import org.apache.kafka.common.header.internals.RecordHeader;
-import org.apache.nifi.json.JsonRecordSetWriter;
-import org.apache.nifi.json.JsonTreeReader;
-import org.apache.nifi.kafka.shared.property.PublishStrategy;
 import org.apache.nifi.kafka.shared.transaction.TransactionIdSupplier;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.reporting.InitializationException;
-import org.apache.nifi.serialization.RecordReaderFactory;
-import org.apache.nifi.serialization.RecordSetWriterFactory;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
@@ -57,12 +52,15 @@ import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
 import java.util.regex.Pattern;
+import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
@@ -79,30 +77,29 @@ public class TestPublishKafkaMockParameterized {
 
     public static Stream<Arguments> testCaseParametersProvider() {
         return Stream.of(
-                arguments("PublishRecord/parameterized/flowfileInput1.json",
-                        "account", ".*A.", getAttributes(), PublishStrategy.USE_VALUE,
-                        "PublishRecord/parameterized/kafkaOutput1V.json"),
-                arguments("PublishRecord/parameterized/flowfileInput1.json",
-                        "account", ".*B.", getAttributes(), PublishStrategy.USE_WRAPPER,
-                        "PublishRecord/parameterized/kafkaOutput1W.json"),
-                arguments("PublishRecord/parameterized/flowfileInputA.json",
-                        "key", ".*1", getAttributes(), PublishStrategy.USE_VALUE,
-                        "PublishRecord/parameterized/kafkaOutputAV.json"),
-                arguments("PublishRecord/parameterized/flowfileInputA.json",
-                        "key", ".*2", getAttributes(), PublishStrategy.USE_WRAPPER,
-                        "PublishRecord/parameterized/kafkaOutputAW.json")
+                arguments("Publish/parameterized/flowfileInput1.json",
+                        "key1A", ".*A.", getAttributes(),
+                        "Publish/parameterized/kafkaOutput1A.json"),
+                arguments("Publish/parameterized/flowfileInput1.json",
+                        "key1B", ".*B.", getAttributes(),
+                        "Publish/parameterized/kafkaOutput1B.json"),
+                arguments("Publish/parameterized/flowfileInputA.json",
+                        "keyA1", ".*1", getAttributes(),
+                        "Publish/parameterized/kafkaOutputA1.json"),
+                arguments("Publish/parameterized/flowfileInputA.json",
+                        "keyA2", ".*2", getAttributes(),
+                        "Publish/parameterized/kafkaOutputA2.json")
         );
     }
 
     @ParameterizedTest
     @MethodSource("testCaseParametersProvider")
-    public void testPublishKafkaRecord(final String flowfileInputResource,
-                                       final String messageKeyField,
-                                       final String attributeNameRegex,
-                                       final Map<String, String> attributes,
-                                       final PublishStrategy publishStrategy,
-                                       final String kafkaRecordExpectedOutputResource)
-            throws IOException, InitializationException {
+    public void testPublishKafka(final String flowfileInputResource,
+                                 final String messageKey,
+                                 final String attributeNameRegex,
+                                 final Map<String, String> attributes,
+                                 final String kafkaRecordExpectedOutputResource)
+            throws IOException {
         final byte[] flowfileData = IOUtils.toByteArray(Objects.requireNonNull(
                 getClass().getClassLoader().getResource(flowfileInputResource)));
         logger.trace(new String(flowfileData, UTF_8));
@@ -114,12 +111,11 @@ public class TestPublishKafkaMockParameterized {
         final TestRunner runner = getTestRunner(producedRecords);
         runner.setProperty("topic", "test-topic");
         runner.setProperty("attribute-name-regex", attributeNameRegex);
-        runner.setProperty("message-key-field", messageKeyField);
-        runner.setProperty("publish-strategy", publishStrategy.name());
+        runner.setProperty("kafka-key", messageKey);
         runner.enqueue(flowFile);
         runner.run(1);
         // verify results
-        runner.assertAllFlowFilesTransferred(PublishKafkaRecord_2_6.REL_SUCCESS, 1);
+        runner.assertAllFlowFilesTransferred(PublishKafka_2_6.REL_SUCCESS, 1);
         assertEquals(1, producedRecords.size());
         final ProducerRecord<byte[], byte[]> kafkaRecord = producedRecords.iterator().next();
         final DefaultPrettyPrinter prettyPrinter = new DefaultPrettyPrinter()
@@ -151,7 +147,7 @@ public class TestPublishKafkaMockParameterized {
         public void serialize(final RecordHeader recordHeader, final JsonGenerator jsonGenerator,
                               final SerializerProvider serializerProvider) throws IOException {
             jsonGenerator.writeStartObject();
-            jsonGenerator.writeObjectField("RecordHeader-key",
+            jsonGenerator.writeStringField("RecordHeader-key",
                     (recordHeader.key() == null) ? null : recordHeader.key());
             jsonGenerator.writeObjectField("RecordHeader-value",
                     (recordHeader.value() == null) ? null : new String(recordHeader.value(), StandardCharsets.UTF_8));
@@ -174,11 +170,15 @@ public class TestPublishKafkaMockParameterized {
         public void serialize(ProducerRecord<byte[], byte[]> producerRecord, JsonGenerator jsonGenerator,
                               SerializerProvider serializerProvider) throws IOException {
             jsonGenerator.writeStartObject();
-            jsonGenerator.writeObjectField("ProducerRecord-key",
-                    (producerRecord.key() == null) ? null : objectMapper.readTree(producerRecord.key()));
+            jsonGenerator.writeStringField("ProducerRecord-key",
+                    (producerRecord.key() == null) ? null : new String(producerRecord.key(), StandardCharsets.UTF_8));
             jsonGenerator.writeObjectField("ProducerRecord-value",
                     (producerRecord.value() == null) ? null : objectMapper.readTree(producerRecord.value()));
-            jsonGenerator.writeObjectField("ProducerRecord-headers", producerRecord.headers());
+            final List<Header> headers = new ArrayList<>();
+            producerRecord.headers().forEach(headers::add);
+            final List<Header> headersSorted = headers.stream()
+                    .sorted(Comparator.comparing(Header::key)).collect(Collectors.toList());
+            jsonGenerator.writeObjectField("ProducerRecord-headers", headersSorted);
             jsonGenerator.writeEndObject();
         }
     }
@@ -192,15 +192,8 @@ public class TestPublishKafkaMockParameterized {
         return attributes;
     }
 
-    private TestRunner getTestRunner(final Collection<ProducerRecord<byte[], byte[]>> producedRecords)
-            throws InitializationException {
-        final String readerId = "record-reader";
-        final RecordReaderFactory readerService = new JsonTreeReader();
-        final String writerId = "record-writer";
-        final RecordSetWriterFactory writerService = new JsonRecordSetWriter();
-        final String keyWriterId = "record-key-writer";
-        final RecordSetWriterFactory keyWriterService = new JsonRecordSetWriter();
-        final PublishKafkaRecord_2_6 processor = new PublishKafkaRecord_2_6() {
+    private TestRunner getTestRunner(final Collection<ProducerRecord<byte[], byte[]>> producedRecords) {
+        final PublishKafka_2_6 processor = new PublishKafka_2_6() {
             @Override
             protected PublisherPool createPublisherPool(final ProcessContext context) {
                 return getPublisherPool(producedRecords, context);
@@ -208,15 +201,6 @@ public class TestPublishKafkaMockParameterized {
         };
         final TestRunner runner = TestRunners.newTestRunner(processor);
         runner.setValidateExpressionUsage(false);
-        runner.addControllerService(readerId, readerService);
-        runner.enableControllerService(readerService);
-        runner.setProperty(readerId, readerId);
-        runner.addControllerService(writerId, writerService);
-        runner.enableControllerService(writerService);
-        runner.setProperty(writerId, writerId);
-        runner.addControllerService(keyWriterId, keyWriterService);
-        runner.enableControllerService(keyWriterService);
-        runner.setProperty(keyWriterId, keyWriterId);
         return runner;
     }
 
@@ -229,10 +213,8 @@ public class TestPublishKafkaMockParameterized {
         final boolean useTransactions = context.getProperty("use-transactions").asBoolean();
         final String transactionalIdPrefix = context.getProperty("transactional-id-prefix").evaluateAttributeExpressions().getValue();
         Supplier<String> transactionalIdSupplier = new TransactionIdSupplier(transactionalIdPrefix);
-        final PublishStrategy publishStrategy = PublishStrategy.valueOf(context.getProperty("publish-strategy").getValue());
         final String charsetName = context.getProperty("message-header-encoding").evaluateAttributeExpressions().getValue();
         final Charset charset = Charset.forName(charsetName);
-        final RecordSetWriterFactory recordKeyWriterFactory = context.getProperty("record-key-writer").asControllerService(RecordSetWriterFactory.class);
 
         return new PublisherPool(
                 Collections.emptyMap(),
@@ -243,8 +225,8 @@ public class TestPublishKafkaMockParameterized {
                 transactionalIdSupplier,
                 attributeNamePattern,
                 charset,
-                publishStrategy,
-                recordKeyWriterFactory) {
+                null,
+                null) {
             @Override
             public PublisherLease obtainPublisher() {
                 return getPublisherLease(producedRecords, context);
@@ -259,9 +241,6 @@ public class TestPublishKafkaMockParameterized {
                                              final ProcessContext context) {
         final String attributeNameRegex = context.getProperty("attribute-name-regex").getValue();
         final Pattern patternAttributeName = (attributeNameRegex == null) ? null : Pattern.compile(attributeNameRegex);
-        final RecordSetWriterFactory keyWriterFactory = context.getProperty("record-key-writer")
-                .asControllerService(RecordSetWriterFactory.class);
-        final PublishStrategy publishStrategy = PublishStrategy.valueOf(context.getProperty("publish-strategy").getValue());
 
         final Producer<byte[], byte[]> producer = mock(ProducerBB.class);
         when(producer.send(any(), any())).then(invocation -> {
@@ -280,8 +259,8 @@ public class TestPublishKafkaMockParameterized {
                 true,
                 patternAttributeName,
                 UTF_8,
-                publishStrategy,
-                keyWriterFactory) {
+                null,
+                null) {
             @Override
             protected long getTimestamp() {
                 return 1000000000000L;
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaMockParameterized.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaRecordMockParameterized.java
similarity index 86%
copy from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaMockParameterized.java
copy to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaRecordMockParameterized.java
index 4e97a6da84..f5785ac803 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaMockParameterized.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaRecordMockParameterized.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processors.kafka.pubsub;
 
 import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParseException;
 import com.fasterxml.jackson.core.util.DefaultIndenter;
 import com.fasterxml.jackson.core.util.DefaultPrettyPrinter;
 import com.fasterxml.jackson.databind.JsonSerializer;
@@ -72,7 +73,7 @@ import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-public class TestPublishKafkaMockParameterized {
+public class TestPublishKafkaRecordMockParameterized {
     private final Logger logger = LoggerFactory.getLogger(getClass());
 
     private final ObjectMapper mapper = getObjectMapper();
@@ -90,7 +91,17 @@ public class TestPublishKafkaMockParameterized {
                         "PublishRecord/parameterized/kafkaOutputAV.json"),
                 arguments("PublishRecord/parameterized/flowfileInputA.json",
                         "key", ".*2", getAttributes(), PublishStrategy.USE_WRAPPER,
-                        "PublishRecord/parameterized/kafkaOutputAW.json")
+                        "PublishRecord/parameterized/kafkaOutputAW.json"),
+
+                arguments("PublishRecord/parameterized/flowfileInputDoc1V.json",
+                        "account", "attribute.*", getAttributesDoc1(), PublishStrategy.USE_VALUE,
+                        "PublishRecord/parameterized/kafkaOutputDoc1V.json"),
+                arguments("PublishRecord/parameterized/flowfileInputDoc1W.json",
+                        null, null, Collections.emptyMap(), PublishStrategy.USE_WRAPPER,
+                        "PublishRecord/parameterized/kafkaOutputDoc1W.json"),
+                arguments("PublishRecord/parameterized/flowfileInputDoc2W.json",
+                        null, null, Collections.emptyMap(), PublishStrategy.USE_WRAPPER,
+                        "PublishRecord/parameterized/kafkaOutputDoc2W.json")
         );
     }
 
@@ -113,8 +124,12 @@ public class TestPublishKafkaMockParameterized {
         final Collection<ProducerRecord<byte[], byte[]>> producedRecords = new ArrayList<>();
         final TestRunner runner = getTestRunner(producedRecords);
         runner.setProperty("topic", "test-topic");
-        runner.setProperty("attribute-name-regex", attributeNameRegex);
-        runner.setProperty("message-key-field", messageKeyField);
+        if (attributeNameRegex != null) {
+            runner.setProperty("attribute-name-regex", attributeNameRegex);
+        }
+        if (messageKeyField != null) {
+            runner.setProperty("message-key-field", messageKeyField);
+        }
         runner.setProperty("publish-strategy", publishStrategy.name());
         runner.enqueue(flowFile);
         runner.run(1);
@@ -174,13 +189,23 @@ public class TestPublishKafkaMockParameterized {
         public void serialize(ProducerRecord<byte[], byte[]> producerRecord, JsonGenerator jsonGenerator,
                               SerializerProvider serializerProvider) throws IOException {
             jsonGenerator.writeStartObject();
-            jsonGenerator.writeObjectField("ProducerRecord-key",
-                    (producerRecord.key() == null) ? null : objectMapper.readTree(producerRecord.key()));
-            jsonGenerator.writeObjectField("ProducerRecord-value",
-                    (producerRecord.value() == null) ? null : objectMapper.readTree(producerRecord.value()));
+            serializeField(jsonGenerator, "ProducerRecord-key", producerRecord.key());
+            serializeField(jsonGenerator, "ProducerRecord-value", producerRecord.value());
             jsonGenerator.writeObjectField("ProducerRecord-headers", producerRecord.headers());
             jsonGenerator.writeEndObject();
         }
+
+        private void serializeField(final JsonGenerator jsonGenerator, final String key, final byte[] value) throws IOException {
+            if (value == null) {
+                jsonGenerator.writeObjectField(key, null);
+            } else {
+                try {
+                    jsonGenerator.writeObjectField(key, objectMapper.readTree(value));
+                } catch (final JsonParseException e) {
+                    jsonGenerator.writeStringField(key, new String(value, UTF_8));
+                }
+            }
+        }
     }
 
     private static Map<String, String> getAttributes() {
@@ -192,6 +217,14 @@ public class TestPublishKafkaMockParameterized {
         return attributes;
     }
 
+    private static Map<String, String> getAttributesDoc1() {
+        final Map<String, String> attributes = new TreeMap<>();
+        attributes.put("attributeA", "valueA");
+        attributes.put("attributeB", "valueB");
+        attributes.put("otherAttribute", "otherValue");
+        return attributes;
+    }
+
     private TestRunner getTestRunner(final Collection<ProducerRecord<byte[], byte[]>> producedRecords)
             throws InitializationException {
         final String readerId = "record-reader";
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/flowfileInput1.json b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/flowfileInput1.json
new file mode 100644
index 0000000000..7764158297
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/flowfileInput1.json
@@ -0,0 +1,8 @@
+{
+  "address": "1234 First Street",
+  "zip": "12345",
+  "account": {
+    "name": "Acme",
+    "number": "AC1234"
+  }
+}
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/flowfileInputA.json b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/flowfileInputA.json
new file mode 100644
index 0000000000..ee19739c31
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/flowfileInputA.json
@@ -0,0 +1,12 @@
+{
+  "key": {
+    "type": "person"
+  },
+  "value": {
+    "name": "Mark",
+    "number": 49
+  },
+  "headers": {
+    "headerA": "headerAValue"
+  }
+}
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/kafkaOutput1A.json b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/kafkaOutput1A.json
new file mode 100644
index 0000000000..1fb83cc47c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/kafkaOutput1A.json
@@ -0,0 +1,18 @@
+{
+  "ProducerRecord-key" : "key1A",
+  "ProducerRecord-value" : {
+    "address" : "1234 First Street",
+    "zip" : "12345",
+    "account" : {
+      "name" : "Acme",
+      "number" : "AC1234"
+    }
+  },
+  "ProducerRecord-headers" : [ {
+    "RecordHeader-key" : "attrKeyA1",
+    "RecordHeader-value" : "attrValueA1"
+  }, {
+    "RecordHeader-key" : "attrKeyA2",
+    "RecordHeader-value" : "attrValueA2"
+  } ]
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/kafkaOutput1B.json b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/kafkaOutput1B.json
new file mode 100644
index 0000000000..49e9804e43
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/kafkaOutput1B.json
@@ -0,0 +1,18 @@
+{
+  "ProducerRecord-key" : "key1B",
+  "ProducerRecord-value" : {
+    "address" : "1234 First Street",
+    "zip" : "12345",
+    "account" : {
+      "name" : "Acme",
+      "number" : "AC1234"
+    }
+  },
+  "ProducerRecord-headers" : [ {
+    "RecordHeader-key" : "attrKeyB1",
+    "RecordHeader-value" : "attrValueB1"
+  }, {
+    "RecordHeader-key" : "attrKeyB2",
+    "RecordHeader-value" : "attrValueB2"
+  } ]
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/kafkaOutputA1.json b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/kafkaOutputA1.json
new file mode 100644
index 0000000000..baa3a1a927
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/kafkaOutputA1.json
@@ -0,0 +1,22 @@
+{
+  "ProducerRecord-key" : "keyA1",
+  "ProducerRecord-value" : {
+    "key" : {
+      "type" : "person"
+    },
+    "value" : {
+      "name" : "Mark",
+      "number" : 49
+    },
+    "headers" : {
+      "headerA" : "headerAValue"
+    }
+  },
+  "ProducerRecord-headers" : [ {
+    "RecordHeader-key" : "attrKeyA1",
+    "RecordHeader-value" : "attrValueA1"
+  }, {
+    "RecordHeader-key" : "attrKeyB1",
+    "RecordHeader-value" : "attrValueB1"
+  } ]
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/kafkaOutputA2.json b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/kafkaOutputA2.json
new file mode 100644
index 0000000000..ba620f70b7
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/Publish/parameterized/kafkaOutputA2.json
@@ -0,0 +1,22 @@
+{
+  "ProducerRecord-key" : "keyA2",
+  "ProducerRecord-value" : {
+    "key" : {
+      "type" : "person"
+    },
+    "value" : {
+      "name" : "Mark",
+      "number" : 49
+    },
+    "headers" : {
+      "headerA" : "headerAValue"
+    }
+  },
+  "ProducerRecord-headers" : [ {
+    "RecordHeader-key" : "attrKeyA2",
+    "RecordHeader-value" : "attrValueA2"
+  }, {
+    "RecordHeader-key" : "attrKeyB2",
+    "RecordHeader-value" : "attrValueB2"
+  } ]
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/flowfileInputDoc1V.json b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/flowfileInputDoc1V.json
new file mode 100644
index 0000000000..7764158297
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/flowfileInputDoc1V.json
@@ -0,0 +1,8 @@
+{
+  "address": "1234 First Street",
+  "zip": "12345",
+  "account": {
+    "name": "Acme",
+    "number": "AC1234"
+  }
+}
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/flowfileInputDoc1W.json b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/flowfileInputDoc1W.json
new file mode 100644
index 0000000000..72db3c15ff
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/flowfileInputDoc1W.json
@@ -0,0 +1,15 @@
+{
+  "key": "Acme Holdings",
+  "value": {
+    "address": "1234 First Street",
+    "zip": "12345",
+    "account": {
+      "name": "Acme",
+      "number":"AC1234"
+    }
+  },
+  "headers": {
+    "accountType": "enterprise",
+    "test": "true"
+  }
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/flowfileInputDoc2W.json b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/flowfileInputDoc2W.json
new file mode 100644
index 0000000000..1a2087851b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/flowfileInputDoc2W.json
@@ -0,0 +1,15 @@
+{
+  "key": {
+    "accountName": "Acme Holdings",
+    "accountHolder": "John Doe",
+    "accountId": "280182830-A009"
+  },
+  "value": {
+    "address": "1234 First Street",
+    "zip": "12345",
+    "account": {
+      "name": "Acme",
+      "number":"AC1234"
+    }
+  }
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/kafkaOutputDoc1V.json b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/kafkaOutputDoc1V.json
new file mode 100644
index 0000000000..146b5fdb00
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/kafkaOutputDoc1V.json
@@ -0,0 +1,21 @@
+{
+  "ProducerRecord-key" : {
+    "name" : "Acme",
+    "number" : "AC1234"
+  },
+  "ProducerRecord-value" : {
+    "address" : "1234 First Street",
+    "zip" : "12345",
+    "account" : {
+      "name" : "Acme",
+      "number" : "AC1234"
+    }
+  },
+  "ProducerRecord-headers" : [ {
+    "RecordHeader-key" : "attributeA",
+    "RecordHeader-value" : "valueA"
+  }, {
+    "RecordHeader-key" : "attributeB",
+    "RecordHeader-value" : "valueB"
+  } ]
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/kafkaOutputDoc1W.json b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/kafkaOutputDoc1W.json
new file mode 100644
index 0000000000..f774579d56
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/kafkaOutputDoc1W.json
@@ -0,0 +1,18 @@
+{
+  "ProducerRecord-key" : "Acme Holdings",
+  "ProducerRecord-value" : {
+    "address" : "1234 First Street",
+    "zip" : "12345",
+    "account" : {
+      "name" : "Acme",
+      "number" : "AC1234"
+    }
+  },
+  "ProducerRecord-headers" : [ {
+    "RecordHeader-key" : "accountType",
+    "RecordHeader-value" : "enterprise"
+  }, {
+    "RecordHeader-key" : "test",
+    "RecordHeader-value" : "true"
+  } ]
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/kafkaOutputDoc2W.json b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/kafkaOutputDoc2W.json
new file mode 100644
index 0000000000..a38fb4e6c1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/resources/PublishRecord/parameterized/kafkaOutputDoc2W.json
@@ -0,0 +1,16 @@
+{
+  "ProducerRecord-key" : {
+    "accountName" : "Acme Holdings",
+    "accountHolder" : "John Doe",
+    "accountId" : "280182830-A009"
+  },
+  "ProducerRecord-value" : {
+    "address" : "1234 First Street",
+    "zip" : "12345",
+    "account" : {
+      "name" : "Acme",
+      "number" : "AC1234"
+    }
+  },
+  "ProducerRecord-headers" : [ ]
+}
\ No newline at end of file


[nifi] 06/22: NIFI-10873 - GenerateFlowFile: flowfiles in a batch are not unique

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit e0b4feac575a9c241b19bacdfd3d166015d43eda
Author: Arpad Boda <ab...@apache.org>
AuthorDate: Thu Nov 24 16:26:31 2022 +0100

    NIFI-10873 - GenerateFlowFile: flowfiles in a batch are not unique
    
    This closes #6717.
    
    Signed-off-by: Peter Turcsanyi <tu...@apache.org>
---
 .../nifi/processors/standard/GenerateFlowFile.java | 22 +++++++++++++---------
 1 file changed, 13 insertions(+), 9 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java
index 0d323e68cf..5c9b7cffd3 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java
@@ -219,13 +219,16 @@ public class GenerateFlowFile extends AbstractProcessor {
     @Override
     public void onTrigger(final ProcessContext context, final ProcessSession session) {
         final byte[] data;
-        if (context.getProperty(UNIQUE_FLOWFILES).asBoolean()) {
-            data = generateData(context);
-        } else if(context.getProperty(CUSTOM_TEXT).isSet()) {
-            final Charset charset = Charset.forName(context.getProperty(CHARSET).getValue());
-            data = context.getProperty(CUSTOM_TEXT).evaluateAttributeExpressions().getValue().getBytes(charset);
+        final boolean uniqueData = context.getProperty(UNIQUE_FLOWFILES).asBoolean();
+        if (uniqueData) {
+            data = new byte[0];
         } else {
-            data = this.data.get();
+            if (context.getProperty(CUSTOM_TEXT).isSet()) {
+                final Charset charset = Charset.forName(context.getProperty(CHARSET).getValue());
+                data = context.getProperty(CUSTOM_TEXT).evaluateAttributeExpressions().getValue().getBytes(charset);
+            } else {
+                data = this.data.get();
+            }
         }
 
         Map<PropertyDescriptor, String> processorProperties = context.getProperties();
@@ -243,12 +246,13 @@ public class GenerateFlowFile extends AbstractProcessor {
         }
 
         for (int i = 0; i < context.getProperty(BATCH_SIZE).asInteger(); i++) {
-            FlowFile flowFile = session.create();
-            if (data.length > 0) {
+        FlowFile flowFile = session.create();
+            final byte[] writtenData = uniqueData ? generateData(context) : data;
+            if (writtenData.length > 0) {
                 flowFile = session.write(flowFile, new OutputStreamCallback() {
                     @Override
                     public void process(final OutputStream out) throws IOException {
-                        out.write(data);
+                        out.write(writtenData);
                     }
                 });
             }


[nifi] 07/22: NIFI-10874 Fixed issue with multiple levels of versioned flows

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 69e47866668ffdf455f585c5f7b987353b87cbda
Author: Bence Simon <bs...@apache.org>
AuthorDate: Thu Nov 24 21:41:50 2022 +0100

    NIFI-10874 Fixed issue with multiple levels of versioned flows
    
    This closes #6719
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 .../java/org/apache/nifi/web/api/ProcessGroupResource.java    | 11 +++++++++--
 .../src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java |  2 ++
 2 files changed, 11 insertions(+), 2 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
index 6704ed5678..dee9a7e9bd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
@@ -51,6 +51,7 @@ import org.apache.nifi.connectable.ConnectableType;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.serialization.FlowEncodingVersion;
 import org.apache.nifi.controller.service.ControllerServiceState;
+import org.apache.nifi.flow.VersionedFlowCoordinates;
 import org.apache.nifi.flow.VersionedParameterContext;
 import org.apache.nifi.flow.VersionedProcessGroup;
 import org.apache.nifi.parameter.ParameterContext;
@@ -2013,7 +2014,13 @@ public class ProcessGroupResource extends FlowUpdateResource<ProcessGroupImportE
             // Step 2: Retrieve flow from Flow Registry
             final RegisteredFlowSnapshot flowSnapshot = getFlowFromRegistry(versionControlInfo);
 
-            // Step 3: Resolve Bundle info
+            // Step 3: Enrich version control info came from UI
+            if (flowSnapshot.getFlowContents() != null && flowSnapshot.getFlowContents().getVersionedFlowCoordinates() != null) {
+                final VersionedFlowCoordinates versionedFlowCoordinates = flowSnapshot.getFlowContents().getVersionedFlowCoordinates();
+                flowSnapshot.getFlowContents().getVersionedFlowCoordinates().setStorageLocation(versionedFlowCoordinates.getStorageLocation());
+            }
+
+            // Step 4: Resolve Bundle info
             serviceFacade.discoverCompatibleBundles(flowSnapshot.getFlowContents());
 
             // If there are any Controller Services referenced that are inherited from the parent group, resolve those to point to the appropriate Controller Service, if we are able to.
@@ -2022,7 +2029,7 @@ public class ProcessGroupResource extends FlowUpdateResource<ProcessGroupImportE
             // If there are any Parameter Providers referenced by Parameter Contexts, resolve these to point to the appropriate Parameter Provider, if we are able to.
             serviceFacade.resolveParameterProviders(flowSnapshot, NiFiUserUtils.getNiFiUser());
 
-            // Step 4: Update contents of the ProcessGroupDTO passed in to include the components that need to be added.
+            // Step 5: Update contents of the ProcessGroupDTO passed in to include the components that need to be added.
             requestProcessGroupEntity.setVersionedFlowSnapshot(flowSnapshot);
         }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index c2421b54f4..d28f9d93a2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -2810,6 +2810,7 @@ public final class DtoFactory {
         dto.setFlowName(versionControlInfo.getFlowName());
         dto.setFlowDescription(versionControlInfo.getFlowDescription());
         dto.setVersion(versionControlInfo.getVersion());
+        dto.setStorageLocation(versionControlInfo.getStorageLocation());
 
         final VersionedFlowStatus status = versionControlInfo.getStatus();
         final VersionedFlowState state = status.getState();
@@ -4594,6 +4595,7 @@ public final class DtoFactory {
         copy.setVersion(original.getVersion());
         copy.setState(original.getState());
         copy.setStateExplanation(original.getStateExplanation());
+        copy.setStorageLocation(original.getStorageLocation());
         return copy;
     }
 


[nifi] 10/22: NIFI-7190 CaptureChangeMySQL processor remove comments from normalized query

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 0ed8114dcb05eefecb8f6c1234449f6f1a362d47
Author: Vasily Makarov <fe...@gmail.com>
AuthorDate: Fri Jun 26 10:44:29 2020 +0300

    NIFI-7190 CaptureChangeMySQL processor remove comments from normalized query
    
    This closes #6711
    
    Co-authored-by: Vasily Makarov <fe...@gmail.com>
    Co-authored-by: Matt Burgess <ma...@apache.org>
    Signed-off-by: David Handermann <ex...@apache.org>
---
 .../cdc/mysql/processors/CaptureChangeMySQL.java   |  16 +-
 .../mysql/processors/CaptureChangeMySQLTest.groovy |  35 ++
 .../nifi/cdc/mysql/CaptureChangeMySQLTest.java     | 393 ---------------------
 .../nifi/cdc/mysql/MockBinlogClientJava.java       | 108 ------
 4 files changed, 50 insertions(+), 502 deletions(-)

diff --git a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/main/java/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQL.java b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/main/java/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQL.java
index 3e245c7ab0..08dad6dc40 100644
--- a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/main/java/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQL.java
+++ b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/main/java/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQL.java
@@ -157,6 +157,9 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
     // Random invalid constant used as an indicator to not set the binlog position on the client (thereby using the latest available)
     private static final int DO_NOT_SET = -1000;
 
+    // A regular expression matching multiline comments, used when parsing DDL statements
+    private static final Pattern MULTI_COMMENT_PATTERN = Pattern.compile("/\\*.*?\\*/", Pattern.DOTALL);
+
     // Relationships
     public static final Relationship REL_SUCCESS = new Relationship.Builder()
             .name("success")
@@ -968,7 +971,8 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
                         currentTable = null;
                     } else {
                         // Check for DDL events (alter table, e.g.). Normalize the query to do string matching on the type of change
-                        String normalizedQuery = sql.toLowerCase().trim().replaceAll(" {2,}", " ");
+                        String normalizedQuery = normalizeQuery(sql);
+
                         if (normalizedQuery.startsWith("alter table")
                                 || normalizedQuery.startsWith("alter ignore table")
                                 || normalizedQuery.startsWith("create table")
@@ -1111,6 +1115,16 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
         currentSession.clearState(Scope.CLUSTER);
     }
 
+    protected String normalizeQuery(String sql) {
+        String normalizedQuery = sql.toLowerCase().trim().replaceAll(" {2,}", " ");
+
+        //Remove comments from the query
+        normalizedQuery = MULTI_COMMENT_PATTERN.matcher(normalizedQuery).replaceAll("").trim();
+        normalizedQuery = normalizedQuery.replaceAll("#.*", "");
+        normalizedQuery = normalizedQuery.replaceAll("-{2}.*", "");
+        return normalizedQuery;
+    }
+
     protected void stop() throws CDCException {
         try {
             if (binlogClient != null) {
diff --git a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/groovy/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQLTest.groovy b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/groovy/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQLTest.groovy
index 37231e533e..cc03e2aaa6 100644
--- a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/groovy/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQLTest.groovy
+++ b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/groovy/org/apache/nifi/cdc/mysql/processors/CaptureChangeMySQLTest.groovy
@@ -1185,6 +1185,40 @@ class CaptureChangeMySQLTest {
         )
     }
 
+    @Test
+    void testGetXIDEvents() throws Exception {
+        testRunner.setProperty(CaptureChangeMySQL.DRIVER_LOCATION, DRIVER_LOCATION)
+        testRunner.setProperty(CaptureChangeMySQL.HOSTS, "localhost:3306")
+        testRunner.setProperty(CaptureChangeMySQL.USERNAME, "root")
+        testRunner.setProperty(CaptureChangeMySQL.CONNECT_TIMEOUT, "2 seconds")
+        testRunner.setProperty(CaptureChangeMySQL.INCLUDE_BEGIN_COMMIT, "true")
+        final DistributedMapCacheClientImpl cacheClient = createCacheClient()
+        Map<String, String> clientProperties = new HashMap<>()
+        clientProperties.put(DistributedMapCacheClientService.HOSTNAME.getName(), "localhost")
+        testRunner.addControllerService("client", cacheClient, clientProperties)
+        testRunner.setProperty(CaptureChangeMySQL.DIST_CACHE_CLIENT, "client")
+        testRunner.enableControllerService(cacheClient)
+
+        testRunner.run(1, false, true)
+        // COMMIT
+        EventHeaderV4 header2 = new EventHeaderV4()
+        header2.setEventType(EventType.XID)
+        header2.setNextPosition(12)
+        header2.setTimestamp(new Date().getTime())
+        EventData eventData = new EventData() {
+        };
+        client.sendEvent(new Event(header2, eventData));
+
+        // when we ge a xid event without having got a 'begin' event ,throw an exception
+        assertThrows(AssertionError.class, () -> testRunner.run(1, false, false))
+    }
+
+    @Test
+    void testNormalizeQuery() throws Exception {
+        assertEquals("alter table", processor.normalizeQuery(" alter table"))
+        assertEquals("alter table", processor.normalizeQuery(" /* This is a \n multiline comment test */ alter table"))
+    }
+
     /********************************
      * Mock and helper classes below
      ********************************/
@@ -1224,6 +1258,7 @@ class CaptureChangeMySQLTest {
             when(mockStatement.executeQuery(anyString())).thenReturn(mockResultSet)
             return mockConnection
         }
+
     }
 
 
diff --git a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/java/org/apache/nifi/cdc/mysql/CaptureChangeMySQLTest.java b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/java/org/apache/nifi/cdc/mysql/CaptureChangeMySQLTest.java
deleted file mode 100644
index 0d3fd0f257..0000000000
--- a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/java/org/apache/nifi/cdc/mysql/CaptureChangeMySQLTest.java
+++ /dev/null
@@ -1,393 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.cdc.mysql;
-
-
-import com.github.shyiko.mysql.binlog.BinaryLogClient;
-import com.github.shyiko.mysql.binlog.event.Event;
-import com.github.shyiko.mysql.binlog.event.EventData;
-import com.github.shyiko.mysql.binlog.event.EventHeaderV4;
-import com.github.shyiko.mysql.binlog.event.EventType;
-import com.github.shyiko.mysql.binlog.event.QueryEventData;
-import com.github.shyiko.mysql.binlog.event.RotateEventData;
-import com.github.shyiko.mysql.binlog.network.SSLMode;
-import org.apache.commons.io.output.WriterOutputStream;
-import org.apache.nifi.cdc.event.ColumnDefinition;
-import org.apache.nifi.cdc.event.TableInfo;
-import org.apache.nifi.cdc.event.TableInfoCacheKey;
-import org.apache.nifi.cdc.mysql.processors.CaptureChangeMySQL;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.controller.AbstractControllerService;
-import org.apache.nifi.distributed.cache.client.Deserializer;
-import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
-import org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService;
-import org.apache.nifi.distributed.cache.client.Serializer;
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.reporting.InitializationException;
-import org.apache.nifi.state.MockStateManager;
-import org.apache.nifi.util.MockComponentLog;
-import org.apache.nifi.util.MockControllerServiceInitializationContext;
-import org.apache.nifi.util.TestRunner;
-import org.apache.nifi.util.TestRunners;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.nio.charset.StandardCharsets;
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeoutException;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.mockito.ArgumentMatchers.anyString;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public class CaptureChangeMySQLTest {
-
-    private static final String DRIVER_LOCATION = "http://mysql-driver.com/driver.jar";
-    CaptureChangeMySQL processor;
-    TestRunner testRunner;
-    MockBinlogClientJava client = new MockBinlogClientJava("localhost", 3306, "root", "password");
-
-    @BeforeEach
-    void setUp() throws Exception {
-        processor = new MockCaptureChangeMySQL();
-        testRunner = TestRunners.newTestRunner(processor);
-    }
-
-    @Test
-    void testConnectionFailures() throws Exception {
-        testRunner.setProperty(CaptureChangeMySQL.DRIVER_LOCATION, DRIVER_LOCATION);
-        testRunner.setProperty(CaptureChangeMySQL.HOSTS, "localhost:3306");
-        testRunner.setProperty(CaptureChangeMySQL.USERNAME, "root");
-        testRunner.setProperty(CaptureChangeMySQL.SERVER_ID, "1");
-        final DistributedMapCacheClientImpl cacheClient = createCacheClient();
-        Map<String, String> clientProperties = new HashMap<>();
-        clientProperties.put(DistributedMapCacheClientService.HOSTNAME.getName(), "localhost");
-        testRunner.addControllerService("client", cacheClient, clientProperties);
-        testRunner.setProperty(CaptureChangeMySQL.DIST_CACHE_CLIENT, "client");
-        testRunner.enableControllerService(cacheClient);
-        client.connectionError = true;
-        try {
-            testRunner.run();
-        } catch (AssertionError ae) {
-            Throwable pe = ae.getCause();
-            assertTrue(pe instanceof ProcessException);
-            Throwable ioe = pe.getCause();
-            assertTrue(ioe instanceof IOException);
-            assertEquals("Could not connect binlog client to any of the specified hosts due to: Error during connect", ioe.getMessage());
-            assertTrue(ioe.getCause() instanceof IOException);
-        }
-        client.connectionError = false;
-
-        client.connectionTimeout = true;
-        try {
-            testRunner.run();
-        } catch (AssertionError ae) {
-            Throwable pe = ae.getCause();
-            assertTrue(pe instanceof ProcessException);
-            Throwable ioe = pe.getCause();
-            assertTrue(ioe instanceof IOException);
-            assertEquals("Could not connect binlog client to any of the specified hosts due to: Connection timed out", ioe.getMessage());
-            assertTrue(ioe.getCause() instanceof TimeoutException);
-        }
-        client.connectionTimeout = false;
-    }
-
-    @Test
-    void testSslModeDisabledSslContextServiceNotRequired() {
-        testRunner.setProperty(CaptureChangeMySQL.HOSTS, "localhost:3306");
-        testRunner.setProperty(CaptureChangeMySQL.SSL_MODE, SSLMode.DISABLED.toString());
-        testRunner.assertValid();
-    }
-
-    @Test
-    void testGetXIDEvents() throws Exception {
-        testRunner.setProperty(CaptureChangeMySQL.DRIVER_LOCATION, DRIVER_LOCATION);
-        testRunner.setProperty(CaptureChangeMySQL.HOSTS, "localhost:3306");
-        testRunner.setProperty(CaptureChangeMySQL.USERNAME, "root");
-        testRunner.setProperty(CaptureChangeMySQL.CONNECT_TIMEOUT, "2 seconds");
-        testRunner.setProperty(CaptureChangeMySQL.INCLUDE_BEGIN_COMMIT, "true");
-        final DistributedMapCacheClientImpl cacheClient = createCacheClient();
-        Map<String, String> clientProperties = new HashMap<>();
-        clientProperties.put(DistributedMapCacheClientService.HOSTNAME.getName(), "localhost");
-        testRunner.addControllerService("client", cacheClient, clientProperties);
-        testRunner.setProperty(CaptureChangeMySQL.DIST_CACHE_CLIENT, "client");
-        testRunner.enableControllerService(cacheClient);
-
-        testRunner.run(1, false, true);
-        // COMMIT
-        EventHeaderV4 header2 = new EventHeaderV4();
-        header2.setEventType(EventType.XID);
-        header2.setNextPosition(12);
-        header2.setTimestamp(new Date().getTime());
-        EventData eventData = new EventData() {
-        };
-        client.sendEvent(new Event(header2, eventData));
-
-        // when we ge a xid event without having got a 'begin' event ,throw an exception
-        assertThrows(AssertionError.class, () -> testRunner.run(1, false, false));
-    }
-
-    @Test
-    void testBeginCommitTransaction() throws Exception {
-        testRunner.setProperty(CaptureChangeMySQL.DRIVER_LOCATION, DRIVER_LOCATION);
-        testRunner.setProperty(CaptureChangeMySQL.HOSTS, "localhost:3306");
-        testRunner.setProperty(CaptureChangeMySQL.USERNAME, "root");
-        testRunner.setProperty(CaptureChangeMySQL.CONNECT_TIMEOUT, "2 seconds");
-        testRunner.setProperty(CaptureChangeMySQL.INCLUDE_BEGIN_COMMIT, "true");
-        final DistributedMapCacheClientImpl cacheClient = createCacheClient();
-        Map<String, String> clientProperties = new HashMap<>();
-        clientProperties.put(DistributedMapCacheClientService.HOSTNAME.getName(), "localhost");
-        testRunner.addControllerService("client", cacheClient, clientProperties);
-        testRunner.setProperty(CaptureChangeMySQL.DIST_CACHE_CLIENT, "client");
-        testRunner.enableControllerService(cacheClient);
-
-
-        testRunner.run(1, false, true);
-
-        EventHeaderV4 header = new EventHeaderV4();
-        header.setEventType(EventType.ROTATE);
-        header.setNextPosition(2);
-        header.setTimestamp(new Date().getTime());
-        RotateEventData rotateEventData = new RotateEventData();
-        rotateEventData.setBinlogFilename("mysql-bin.000001");
-        rotateEventData.setBinlogPosition(4L);
-        client.sendEvent(new Event(header, rotateEventData));
-
-        // BEGIN
-        EventHeaderV4 header1 = new EventHeaderV4();
-        header1.setEventType(EventType.QUERY);
-        header1.setNextPosition(6);
-        header1.setTimestamp(new Date().getTime());
-        QueryEventData rotateEventData1 = new QueryEventData();
-        rotateEventData1.setDatabase("mysql-bin.000001");
-        rotateEventData1.setDatabase("myDB");
-        rotateEventData1.setSql("BEGIN");
-        client.sendEvent(new Event(header1, rotateEventData1));
-
-        // COMMIT
-        EventHeaderV4 header2 = new EventHeaderV4();
-        header2.setEventType(EventType.XID);
-        header2.setNextPosition(12);
-        header2.setTimestamp(new Date().getTime());
-        EventData eventData2 = new EventData() {
-        };
-        client.sendEvent(new Event(header2, eventData2));
-
-        //when get a xid event,stop and restart the processor
-        //here we used to get an exception
-        testRunner.run(1, true, false);
-        testRunner.run(1, false, false);
-
-        // next transaction
-        // BEGIN
-        EventHeaderV4 header3 = new EventHeaderV4();
-        header3.setEventType(EventType.QUERY);
-        header3.setNextPosition(16);
-        header3.setTimestamp(new Date().getTime());
-        QueryEventData rotateEventData3 = new QueryEventData();
-        rotateEventData3.setDatabase("mysql-bin.000001");
-        rotateEventData3.setDatabase("myDB");
-        rotateEventData3.setSql("BEGIN");
-        client.sendEvent(new Event(header3, rotateEventData3));
-
-
-        testRunner.run(1, true, false);
-    }
-
-    /********************************
-     * Mock and helper classes below
-     ********************************/
-
-    static DistributedMapCacheClientImpl createCacheClient() throws InitializationException {
-
-        final DistributedMapCacheClientImpl client = new DistributedMapCacheClientImpl();
-        final ComponentLog logger = new MockComponentLog("client", client);
-        final MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client", logger, new MockStateManager(client));
-
-        client.initialize(clientInitContext);
-
-        return client;
-    }
-
-    static final class DistributedMapCacheClientImpl extends AbstractControllerService implements DistributedMapCacheClient {
-
-        private final Map<String, String> cacheMap = new HashMap<>();
-
-
-        @Override
-        protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-            List<PropertyDescriptor> descriptors = new ArrayList<>();
-            descriptors.add(DistributedMapCacheClientService.HOSTNAME);
-            descriptors.add(DistributedMapCacheClientService.COMMUNICATIONS_TIMEOUT);
-            descriptors.add(DistributedMapCacheClientService.PORT);
-            descriptors.add(DistributedMapCacheClientService.SSL_CONTEXT_SERVICE);
-            return descriptors;
-        }
-
-        @Override
-        public <K, V> boolean putIfAbsent(
-                final K key,
-                final V value,
-                final Serializer<K> keySerializer, final Serializer<V> valueSerializer) throws IOException {
-
-            StringWriter keyWriter = new StringWriter();
-            keySerializer.serialize(key, new WriterOutputStream(keyWriter));
-            String keyString = keyWriter.toString();
-
-            if (cacheMap.containsKey(keyString)) return false;
-
-            StringWriter valueWriter = new StringWriter();
-            valueSerializer.serialize(value, new WriterOutputStream(valueWriter));
-            return true;
-        }
-
-        @Override
-        @SuppressWarnings("unchecked")
-        public <K, V> V getAndPutIfAbsent(
-                final K key, final V value, final Serializer<K> keySerializer, final Serializer<V> valueSerializer,
-                final Deserializer<V> valueDeserializer) throws IOException {
-            StringWriter keyWriter = new StringWriter();
-            keySerializer.serialize(key, new WriterOutputStream(keyWriter));
-            String keyString = keyWriter.toString();
-
-            if (cacheMap.containsKey(keyString))
-                return valueDeserializer.deserialize(cacheMap.get(keyString).getBytes(StandardCharsets.UTF_8));
-
-            StringWriter valueWriter = new StringWriter();
-            valueSerializer.serialize(value, new WriterOutputStream(valueWriter));
-            return null;
-        }
-
-        @Override
-        public <K> boolean containsKey(final K key, final Serializer<K> keySerializer) throws IOException {
-            StringWriter keyWriter = new StringWriter();
-            keySerializer.serialize(key, new WriterOutputStream(keyWriter));
-            String keyString = keyWriter.toString();
-
-            return cacheMap.containsKey(keyString);
-        }
-
-        @Override
-        public <K, V> V get(
-                final K key,
-                final Serializer<K> keySerializer, final Deserializer<V> valueDeserializer) throws IOException {
-            StringWriter keyWriter = new StringWriter();
-            keySerializer.serialize(key, new WriterOutputStream(keyWriter));
-            String keyString = keyWriter.toString();
-
-            return (cacheMap.containsKey(keyString)) ? valueDeserializer.deserialize(cacheMap.get(keyString).getBytes(StandardCharsets.UTF_8)) : null;
-        }
-
-        @Override
-        public void close() throws IOException {
-
-        }
-
-        @Override
-        public <K> boolean remove(final K key, final Serializer<K> serializer) throws IOException {
-            StringWriter keyWriter = new StringWriter();
-            serializer.serialize(key, new WriterOutputStream(keyWriter));
-            String keyString = keyWriter.toString();
-
-            boolean removed = (cacheMap.containsKey(keyString));
-            cacheMap.remove(keyString);
-            return removed;
-        }
-
-        @Override
-        public long removeByPattern(String regex) throws IOException {
-            final List<String> removedRecords = new ArrayList<>();
-            Pattern p = Pattern.compile(regex);
-            for (String key : cacheMap.keySet()) {
-                // Key must be backed by something that can be converted into a String
-                Matcher m = p.matcher(key);
-                if (m.matches()) {
-                    removedRecords.add(cacheMap.get(key));
-                }
-            }
-            final long numRemoved = removedRecords.size();
-            for (String it : removedRecords) {
-                cacheMap.remove(it);
-            }
-            return numRemoved;
-        }
-
-        @Override
-        public <K, V> void put(
-                final K key,
-                final V value,
-                final Serializer<K> keySerializer, final Serializer<V> valueSerializer) throws IOException {
-            StringWriter keyWriter = new StringWriter();
-            keySerializer.serialize(key, new WriterOutputStream(keyWriter));
-            StringWriter valueWriter = new StringWriter();
-            valueSerializer.serialize(value, new WriterOutputStream(valueWriter));
-        }
-    }
-
-    public class MockCaptureChangeMySQL extends CaptureChangeMySQL {
-
-        Map<TableInfoCacheKey, TableInfo> cache = new HashMap<>();
-
-        public BinaryLogClient createBinlogClient(String hostname, int port, String username, String password) {
-            return client;
-        }
-
-        @Override
-        public TableInfo loadTableInfo(TableInfoCacheKey key) {
-            TableInfo tableInfo = cache.get(key);
-            if (tableInfo == null) {
-                List<ColumnDefinition> column = new ArrayList<>();
-                column.add(new ColumnDefinition((byte) 4, "id"));
-                column.add(new ColumnDefinition((byte) -4, "string1"));
-
-                tableInfo = new TableInfo(key.getDatabaseName(), key.getTableName(), key.getTableId(), column);
-                cache.put(key, tableInfo);
-            }
-            return tableInfo;
-        }
-
-        @Override
-        protected void registerDriver(String locationString, String drvName) throws InitializationException {
-        }
-
-        @Override
-        protected Connection getJdbcConnection() throws SQLException {
-            Connection mockConnection = mock(Connection.class);
-            Statement mockStatement = mock(Statement.class);
-            when(mockConnection.createStatement()).thenReturn(mockStatement);
-            ResultSet mockResultSet = mock(ResultSet.class);
-            when(mockStatement.executeQuery(anyString())).thenReturn(mockResultSet);
-            return mockConnection;
-        }
-    }
-}
diff --git a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/java/org/apache/nifi/cdc/mysql/MockBinlogClientJava.java b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/java/org/apache/nifi/cdc/mysql/MockBinlogClientJava.java
deleted file mode 100644
index d23822292e..0000000000
--- a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/src/test/java/org/apache/nifi/cdc/mysql/MockBinlogClientJava.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.cdc.mysql;
-
-import com.github.shyiko.mysql.binlog.BinaryLogClient;
-import com.github.shyiko.mysql.binlog.event.Event;
-import com.github.shyiko.mysql.binlog.network.SSLSocketFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeoutException;
-
-public class MockBinlogClientJava extends BinaryLogClient {
-    String hostname;
-    int port;
-    String username;
-    String password;
-
-    boolean connected;
-    public boolean connectionTimeout = false;
-    public boolean connectionError = false;
-
-    List<LifecycleListener> lifecycleListeners = new ArrayList<>();
-    SSLSocketFactory sslSocketFactory;
-
-    List<EventListener> eventListeners = new ArrayList<>();
-
-
-    public MockBinlogClientJava(String hostname, int port, String username, String password) {
-        super(hostname, port, username, password);
-        this.hostname = hostname;
-        this.port = port;
-        this.username = username;
-        this.password = password;
-    }
-
-    @Override
-    public void connect(long timeoutInMilliseconds) throws IOException, TimeoutException {
-        if (connectionTimeout) {
-            throw new TimeoutException("Connection timed out");
-        }
-        if (connectionError) {
-            throw new IOException("Error during connect");
-        }
-        if (password == null) {
-            throw new NullPointerException("Password can't be null");
-        }
-        connected = true;
-    }
-
-    @Override
-    public void disconnect() throws IOException {
-        connected = false;
-    }
-
-    @Override
-    public boolean isConnected() {
-        return connected;
-    }
-
-    @Override
-    public void registerEventListener(EventListener eventListener) {
-        eventListeners.add(eventListener);
-    }
-
-    public void unregisterEventListener(EventListener eventListener) {
-        eventListeners.remove(eventListener);
-    }
-
-    @Override
-    public void registerLifecycleListener(LifecycleListener lifecycleListener) {
-        if (!lifecycleListeners.contains(lifecycleListener)) {
-            lifecycleListeners.add(lifecycleListener);
-        }
-    }
-
-    @Override
-    public void unregisterLifecycleListener(LifecycleListener lifecycleListener) {
-        lifecycleListeners.remove(lifecycleListener);
-    }
-
-    @Override
-    public void setSslSocketFactory(SSLSocketFactory sslSocketFactory) {
-        super.setSslSocketFactory(sslSocketFactory);
-        this.sslSocketFactory = sslSocketFactory;
-    }
-
-    public void sendEvent(Event event) {
-        for (EventListener eventListener : eventListeners) {
-            eventListener.onEvent(event);
-        }
-    }
-}


[nifi] 04/22: NIFI-10845 - JsonQueryElasticsearch processors are not outputting an empty flow file for a combined response with output_no_hits set to true

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.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 51395bddd18877b2324a7d9858a8694ea8aa83b0
Author: Ryan Van Den Bos <ry...@naimuri.com>
AuthorDate: Tue Nov 22 09:41:44 2022 +0000

    NIFI-10845 - JsonQueryElasticsearch processors are not outputting an empty flow file for a combined response with output_no_hits set to true
    
    Signed-off-by: Chris Sampson <ch...@gmail.com>
    
    This closes #6701
---
 .../AbstractJsonQueryElasticsearch.java            |  8 +++--
 .../AbstractPaginatedJsonQueryElasticsearch.java   |  3 ++
 ...tractPaginatedJsonQueryElasticsearchTest.groovy | 39 ++++++++++++++++++++++
 3 files changed, 48 insertions(+), 2 deletions(-)

diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractJsonQueryElasticsearch.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractJsonQueryElasticsearch.java
index 1ce0f7f414..f154cf2790 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractJsonQueryElasticsearch.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractJsonQueryElasticsearch.java
@@ -105,6 +105,10 @@ public abstract class AbstractJsonQueryElasticsearch<Q extends JsonQueryParamete
     private String splitUpAggregations;
     private boolean outputNoHits;
 
+    boolean getOutputNoHits() {
+        return outputNoHits;
+    }
+
     final ObjectMapper mapper = new ObjectMapper();
 
     final AtomicReference<ElasticSearchClientService> clientService = new AtomicReference<>(null);
@@ -277,8 +281,8 @@ public abstract class AbstractJsonQueryElasticsearch<Q extends JsonQueryParamete
         }
     }
 
-    private FlowFile writeHitFlowFile(final int count, final String json, final ProcessSession session,
-                                      final FlowFile hitFlowFile, final Map<String, String> attributes) {
+    FlowFile writeHitFlowFile(final int count, final String json, final ProcessSession session,
+                              final FlowFile hitFlowFile, final Map<String, String> attributes) {
         final FlowFile ff = session.write(hitFlowFile, out -> out.write(json.getBytes()));
         attributes.put("hit.count", Integer.toString(count));
 
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractPaginatedJsonQueryElasticsearch.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractPaginatedJsonQueryElasticsearch.java
index 2e8eab6eeb..ec1a020ad7 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractPaginatedJsonQueryElasticsearch.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractPaginatedJsonQueryElasticsearch.java
@@ -273,6 +273,9 @@ public abstract class AbstractPaginatedJsonQueryElasticsearch extends AbstractJs
 
             hitsFlowFiles.add(writeCombinedHitFlowFile(paginatedJsonQueryParameters.getHitCount() + hits.size(),
                     hits, session, hitFlowFile, attributes, append));
+        } else if (getOutputNoHits()) {
+            final FlowFile hitFlowFile = createChildFlowFile(session, parent);
+            hitsFlowFiles.add(writeHitFlowFile(0, "", session, hitFlowFile, attributes));
         }
     }
 
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-processors/src/test/groovy/org/apache/nifi/processors/elasticsearch/AbstractPaginatedJsonQueryElasticsearchTest.groovy b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-processors/src/test/groovy/org/apache/nifi/processors/elasticsearch/AbstractPaginatedJsonQueryElasticsearchTest.groovy
index 274e6a2fd5..0ec4470f76 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-processors/src/test/groovy/org/apache/nifi/processors/elasticsearch/AbstractPaginatedJsonQueryElasticsearchTest.groovy
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-restapi-processors/src/test/groovy/org/apache/nifi/processors/elasticsearch/AbstractPaginatedJsonQueryElasticsearchTest.groovy
@@ -268,4 +268,43 @@ abstract class AbstractPaginatedJsonQueryElasticsearchTest extends AbstractJsonQ
             assertThat(runner.getProvenanceEvents().stream().filter({ pe -> pe.getEventType() == ProvenanceEventType.SEND}).count(), is(0L))
         }
     }
+
+    @Test
+    void testNoHitsFlowFileIsProducedForEachResultSplitSetup() {
+        final TestRunner runner = createRunner(false)
+        final TestElasticsearchClientService service = getService(runner)
+        runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, prettyPrint(toJson([query: [match_all: [:]]])))
+        runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.OUTPUT_NO_HITS, "true")
+        service.setMaxPages(0)
+
+        // test that an empty flow file is produced for a per query setup
+        runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT, AbstractPaginatedJsonQueryElasticsearch.FLOWFILE_PER_QUERY)
+        runOnce(runner)
+        testCounts(runner, isInput() ? 1 : 0, 1, 0, 0)
+
+        runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "0")
+        runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("page.number", "1")
+        runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).getSize() == 0
+        reset(runner)
+
+        // test that an empty flow file is produced for a per hit setup
+        runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT, AbstractPaginatedJsonQueryElasticsearch.FLOWFILE_PER_HIT)
+        runOnce(runner)
+        testCounts(runner, isInput() ? 1 : 0, 1, 0, 0)
+
+        runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "0")
+        runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("page.number", "1")
+        runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).getSize() == 0
+        reset(runner)
+
+        // test that an empty flow file is produced for a per response setup
+        runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT, AbstractPaginatedJsonQueryElasticsearch.FLOWFILE_PER_RESPONSE)
+        runOnce(runner)
+        testCounts(runner, isInput() ? 1 : 0, 1, 0, 0)
+
+        runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "0")
+        runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("page.number", "1")
+        runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).getSize() == 0
+        reset(runner)
+    }
 }