You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ij...@apache.org on 2019/06/07 08:57:26 UTC

[nifi] branch master updated: NIFI-6218 Support setting transactional.id in PublishKafka/PublishKafkaRecord

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

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


The following commit(s) were added to refs/heads/master by this push:
     new aefdc93  NIFI-6218 Support setting transactional.id in PublishKafka/PublishKafkaRecord
aefdc93 is described below

commit aefdc9377cc00c82ab70d44da5effd42da63bb04
Author: Ferenc Szabó <sz...@apache.org>
AuthorDate: Tue Apr 16 16:49:52 2019 +0200

    NIFI-6218 Support setting transactional.id in PublishKafka/PublishKafkaRecord
    
    Added new property to the processors. If it is empty, then the behaviour is the same as before.
    Adde unit test to check if KafkaProducer receives the new property.
    
    Testing Done:
    Unit tests.
    Connecting to kafka and verifying transactionalId-s in kafka server log.
    
    fixing review findings
    
    This closes #3439.
    
    Signed-off-by: Koji Kawamura <ij...@apache.org>
---
 .../kafka/pubsub/KafkaProcessorUtils.java          | 12 +++++++
 .../kafka/pubsub/PublishKafkaRecord_0_11.java      | 14 +++++++-
 .../processors/kafka/pubsub/PublishKafka_0_11.java | 14 +++++++-
 .../processors/kafka/pubsub/PublisherPool.java     |  8 +++--
 .../kafka/pubsub/KafkaProcessorUtilsTest.java      | 41 ++++++++++++++++++++++
 .../processors/kafka/pubsub/TestPublisherPool.java |  6 ++--
 .../kafka/pubsub/KafkaProcessorUtils.java          | 12 +++++++
 .../kafka/pubsub/PublishKafkaRecord_1_0.java       | 14 +++++++-
 .../processors/kafka/pubsub/PublishKafka_1_0.java  | 14 +++++++-
 .../processors/kafka/pubsub/PublisherPool.java     |  8 +++--
 .../kafka/pubsub/KafkaProcessorUtilsTest.java      | 41 ++++++++++++++++++++++
 .../processors/kafka/pubsub/TestPublisherPool.java |  6 ++--
 .../kafka/pubsub/KafkaProcessorUtils.java          | 12 +++++++
 .../kafka/pubsub/PublishKafkaRecord_2_0.java       | 14 +++++++-
 .../processors/kafka/pubsub/PublishKafka_2_0.java  | 14 +++++++-
 .../processors/kafka/pubsub/PublisherPool.java     |  8 +++--
 .../kafka/pubsub/KafkaProcessorUtilsTest.java      | 41 ++++++++++++++++++++++
 .../processors/kafka/pubsub/TestPublisherPool.java |  5 ++-
 18 files changed, 258 insertions(+), 26 deletions(-)

diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
index 37c6599..9bc4917 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
@@ -25,7 +25,9 @@ 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;
 
 import org.apache.commons.lang3.StringUtils;
@@ -332,6 +334,16 @@ final class KafkaProcessorUtils {
     }
 
     /**
+     * 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 />
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_11.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_11.java
index 0815518..cdd37a6 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_11.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_11.java
@@ -62,6 +62,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 
 @Tags({"Apache", "Kafka", "Record", "csv", "json", "avro", "logs", "Put", "Send", "Message", "PubSub", "0.11.x"})
@@ -219,6 +220,14 @@ public class PublishKafkaRecord_0_11 extends AbstractProcessor {
         .defaultValue("true")
         .required(true)
         .build();
+    static final PropertyDescriptor TRANSACTIONAL_ID_PREFIX = new PropertyDescriptor.Builder()
+      .name("transactional-id-prefix")
+      .displayName("Transactional Id Prefix")
+      .description("When Use Transaction is set to true, KafkaProducer config 'transactional.id' will be a generated UUID and will be prefixed with this string.")
+      .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+      .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+      .required(false)
+      .build();
     static final PropertyDescriptor MESSAGE_HEADER_ENCODING = new PropertyDescriptor.Builder()
         .name("message-header-encoding")
         .displayName("Message Header Encoding")
@@ -251,6 +260,7 @@ public class PublishKafkaRecord_0_11 extends AbstractProcessor {
         properties.add(RECORD_READER);
         properties.add(RECORD_WRITER);
         properties.add(USE_TRANSACTIONS);
+        properties.add(TRANSACTIONAL_ID_PREFIX);
         properties.add(DELIVERY_GUARANTEE);
         properties.add(ATTRIBUTE_NAME_REGEX);
         properties.add(MESSAGE_HEADER_ENCODING);
@@ -332,6 +342,8 @@ public class PublishKafkaRecord_0_11 extends AbstractProcessor {
         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);
 
         final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
         final Charset charset = Charset.forName(charsetName);
@@ -342,7 +354,7 @@ public class PublishKafkaRecord_0_11 extends AbstractProcessor {
         kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize));
 
-        return new PublisherPool(kafkaProperties, getLogger(), maxMessageSize, maxAckWaitMillis, useTransactions, attributeNamePattern, charset);
+        return new PublisherPool(kafkaProperties, getLogger(), maxMessageSize, maxAckWaitMillis, useTransactions, transactionalIdSupplier, attributeNamePattern, charset);
     }
 
     @OnStopped
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_11.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_11.java
index 5efe913..78baeec 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_11.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_11.java
@@ -31,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 
 import javax.xml.bind.DatatypeConverter;
@@ -222,6 +223,14 @@ public class PublishKafka_0_11 extends AbstractProcessor {
         .defaultValue("true")
         .required(true)
         .build();
+    static final PropertyDescriptor TRANSACTIONAL_ID_PREFIX = new PropertyDescriptor.Builder()
+        .name("transactional-id-prefix")
+        .displayName("Transactional Id Prefix")
+        .description("When Use Transaction is set to true, KafkaProducer config 'transactional.id' will be a generated UUID and will be prefixed with this string.")
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+        .required(false)
+        .build();
     static final PropertyDescriptor MESSAGE_HEADER_ENCODING = new PropertyDescriptor.Builder()
         .name("message-header-encoding")
         .displayName("Message Header Encoding")
@@ -253,6 +262,7 @@ public class PublishKafka_0_11 extends AbstractProcessor {
         properties.add(TOPIC);
         properties.add(DELIVERY_GUARANTEE);
         properties.add(USE_TRANSACTIONS);
+        properties.add(TRANSACTIONAL_ID_PREFIX);
         properties.add(ATTRIBUTE_NAME_REGEX);
         properties.add(MESSAGE_HEADER_ENCODING);
         properties.add(KEY);
@@ -329,6 +339,8 @@ public class PublishKafka_0_11 extends AbstractProcessor {
         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);
 
         final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
         final Charset charset = Charset.forName(charsetName);
@@ -339,7 +351,7 @@ public class PublishKafka_0_11 extends AbstractProcessor {
         kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize));
 
-        return new PublisherPool(kafkaProperties, getLogger(), maxMessageSize, maxAckWaitMillis, useTransactions, attributeNamePattern, charset);
+        return new PublisherPool(kafkaProperties, getLogger(), maxMessageSize, maxAckWaitMillis, useTransactions, transactionalIdSupplier, attributeNamePattern, charset);
     }
 
     @OnStopped
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
index d5caa8d..54811cc 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
@@ -21,9 +21,9 @@ import java.io.Closeable;
 import java.nio.charset.Charset;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.UUID;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 
 import org.apache.kafka.clients.producer.KafkaProducer;
@@ -39,11 +39,12 @@ public class PublisherPool implements Closeable {
     private final boolean useTransactions;
     private final Pattern attributeNameRegex;
     private final Charset headerCharacterSet;
+    private Supplier<String> transactionalIdSupplier;
 
     private volatile boolean closed = false;
 
     PublisherPool(final Map<String, Object> kafkaProperties, final ComponentLog logger, final int maxMessageSize, final long maxAckWaitMillis,
-        final boolean useTransactions, final Pattern attributeNameRegex, final Charset headerCharacterSet) {
+                  final boolean useTransactions, final Supplier<String> transactionalIdSupplier, final Pattern attributeNameRegex, final Charset headerCharacterSet) {
         this.logger = logger;
         this.publisherQueue = new LinkedBlockingQueue<>();
         this.kafkaProperties = kafkaProperties;
@@ -52,6 +53,7 @@ public class PublisherPool implements Closeable {
         this.useTransactions = useTransactions;
         this.attributeNameRegex = attributeNameRegex;
         this.headerCharacterSet = headerCharacterSet;
+        this.transactionalIdSupplier = transactionalIdSupplier;
     }
 
     public PublisherLease obtainPublisher() {
@@ -71,7 +73,7 @@ public class PublisherPool implements Closeable {
     private PublisherLease createLease() {
         final Map<String, Object> properties = new HashMap<>(kafkaProperties);
         if (useTransactions) {
-            properties.put("transactional.id", UUID.randomUUID().toString());
+            properties.put("transactional.id", transactionalIdSupplier.get());
         }
 
         final Producer<byte[], byte[]> producer = new KafkaProducer<>(properties);
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtilsTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtilsTest.java
new file mode 100644
index 0000000..1a20d2d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtilsTest.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.kafka.pubsub;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.function.Supplier;
+
+import org.junit.Test;
+
+public class KafkaProcessorUtilsTest {
+
+  @Test
+  public void getTransactionalIdSupplierWithPrefix() {
+    Supplier<String> prefix = KafkaProcessorUtils.getTransactionalIdSupplier("prefix");
+    String id = prefix.get();
+    assertTrue(id.startsWith("prefix"));
+    assertEquals(42, id.length());
+  }
+
+  @Test
+  public void getTransactionalIdSupplierWithEmptyPrefix() {
+    Supplier<String> prefix = KafkaProcessorUtils.getTransactionalIdSupplier(null);
+    assertEquals(36, prefix.get().length() );
+  }
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java
index afaf841..4deeba3 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java
@@ -28,7 +28,6 @@ import org.apache.nifi.logging.ComponentLog;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-
 public class TestPublisherPool {
 
     @Test
@@ -38,7 +37,7 @@ public class TestPublisherPool {
         kafkaProperties.put("key.serializer", ByteArraySerializer.class.getName());
         kafkaProperties.put("value.serializer", ByteArraySerializer.class.getName());
 
-        final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L, false, null, StandardCharsets.UTF_8);
+        final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L, false, null, null, StandardCharsets.UTF_8);
         assertEquals(0, pool.available());
 
         final PublisherLease lease = pool.obtainPublisher();
@@ -55,7 +54,7 @@ public class TestPublisherPool {
         kafkaProperties.put("key.serializer", ByteArraySerializer.class.getName());
         kafkaProperties.put("value.serializer", ByteArraySerializer.class.getName());
 
-        final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L, false, null, StandardCharsets.UTF_8);
+        final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L, false, null, null, StandardCharsets.UTF_8);
         assertEquals(0, pool.available());
 
         final PublisherLease lease = pool.obtainPublisher();
@@ -65,5 +64,4 @@ public class TestPublisherPool {
         lease.close();
         assertEquals(0, pool.available());
     }
-
 }
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
index 3cde7ce..7c3c578 100644
--- 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
@@ -25,7 +25,9 @@ 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;
 
 import org.apache.commons.lang3.StringUtils;
@@ -332,6 +334,16 @@ final class KafkaProcessorUtils {
     }
 
     /**
+     * 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 />
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 bb0478c..2a7e7e7 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
@@ -62,6 +62,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 
 @Tags({"Apache", "Kafka", "Record", "csv", "json", "avro", "logs", "Put", "Send", "Message", "PubSub", "1.0"})
@@ -220,6 +221,14 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
         .defaultValue("true")
         .required(true)
         .build();
+    static final PropertyDescriptor TRANSACTIONAL_ID_PREFIX = new PropertyDescriptor.Builder()
+      .name("transactional-id-prefix")
+      .displayName("Transactional Id Prefix")
+      .description("When Use Transaction is set to true, KafkaProducer config 'transactional.id' will be a generated UUID and will be prefixed with this string.")
+      .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+      .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+      .required(false)
+      .build();
     static final PropertyDescriptor MESSAGE_HEADER_ENCODING = new PropertyDescriptor.Builder()
         .name("message-header-encoding")
         .displayName("Message Header Encoding")
@@ -252,6 +261,7 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
         properties.add(RECORD_READER);
         properties.add(RECORD_WRITER);
         properties.add(USE_TRANSACTIONS);
+        properties.add(TRANSACTIONAL_ID_PREFIX);
         properties.add(DELIVERY_GUARANTEE);
         properties.add(ATTRIBUTE_NAME_REGEX);
         properties.add(MESSAGE_HEADER_ENCODING);
@@ -334,6 +344,8 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
         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);
 
         final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
         final Charset charset = Charset.forName(charsetName);
@@ -344,7 +356,7 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
         kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize));
 
-        return new PublisherPool(kafkaProperties, getLogger(), maxMessageSize, maxAckWaitMillis, useTransactions, attributeNamePattern, charset);
+        return new PublisherPool(kafkaProperties, getLogger(), maxMessageSize, maxAckWaitMillis, useTransactions, transactionalIdSupplier, attributeNamePattern, charset);
     }
 
     @OnStopped
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 477aefd..1c3efc0 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,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 
 import javax.xml.bind.DatatypeConverter;
@@ -223,6 +224,14 @@ public class PublishKafka_1_0 extends AbstractProcessor {
         .defaultValue("true")
         .required(true)
         .build();
+    static final PropertyDescriptor TRANSACTIONAL_ID_PREFIX = new PropertyDescriptor.Builder()
+        .name("transactional-id-prefix")
+        .displayName("Transactional Id Prefix")
+        .description("When Use Transaction is set to true, KafkaProducer config 'transactional.id' will be a generated UUID and will be prefixed with this string.")
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+        .required(false)
+        .build();
     static final PropertyDescriptor MESSAGE_HEADER_ENCODING = new PropertyDescriptor.Builder()
         .name("message-header-encoding")
         .displayName("Message Header Encoding")
@@ -254,6 +263,7 @@ public class PublishKafka_1_0 extends AbstractProcessor {
         properties.add(TOPIC);
         properties.add(DELIVERY_GUARANTEE);
         properties.add(USE_TRANSACTIONS);
+        properties.add(TRANSACTIONAL_ID_PREFIX);
         properties.add(ATTRIBUTE_NAME_REGEX);
         properties.add(MESSAGE_HEADER_ENCODING);
         properties.add(KEY);
@@ -331,6 +341,8 @@ public class PublishKafka_1_0 extends AbstractProcessor {
         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);
 
         final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
         final Charset charset = Charset.forName(charsetName);
@@ -341,7 +353,7 @@ public class PublishKafka_1_0 extends AbstractProcessor {
         kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize));
 
-        return new PublisherPool(kafkaProperties, getLogger(), maxMessageSize, maxAckWaitMillis, useTransactions, attributeNamePattern, charset);
+        return new PublisherPool(kafkaProperties, getLogger(), maxMessageSize, maxAckWaitMillis, useTransactions, transactionalIdSupplier, attributeNamePattern, charset);
     }
 
     @OnStopped
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
index d5caa8d..54811cc 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
@@ -21,9 +21,9 @@ import java.io.Closeable;
 import java.nio.charset.Charset;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.UUID;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 
 import org.apache.kafka.clients.producer.KafkaProducer;
@@ -39,11 +39,12 @@ public class PublisherPool implements Closeable {
     private final boolean useTransactions;
     private final Pattern attributeNameRegex;
     private final Charset headerCharacterSet;
+    private Supplier<String> transactionalIdSupplier;
 
     private volatile boolean closed = false;
 
     PublisherPool(final Map<String, Object> kafkaProperties, final ComponentLog logger, final int maxMessageSize, final long maxAckWaitMillis,
-        final boolean useTransactions, final Pattern attributeNameRegex, final Charset headerCharacterSet) {
+                  final boolean useTransactions, final Supplier<String> transactionalIdSupplier, final Pattern attributeNameRegex, final Charset headerCharacterSet) {
         this.logger = logger;
         this.publisherQueue = new LinkedBlockingQueue<>();
         this.kafkaProperties = kafkaProperties;
@@ -52,6 +53,7 @@ public class PublisherPool implements Closeable {
         this.useTransactions = useTransactions;
         this.attributeNameRegex = attributeNameRegex;
         this.headerCharacterSet = headerCharacterSet;
+        this.transactionalIdSupplier = transactionalIdSupplier;
     }
 
     public PublisherLease obtainPublisher() {
@@ -71,7 +73,7 @@ public class PublisherPool implements Closeable {
     private PublisherLease createLease() {
         final Map<String, Object> properties = new HashMap<>(kafkaProperties);
         if (useTransactions) {
-            properties.put("transactional.id", UUID.randomUUID().toString());
+            properties.put("transactional.id", transactionalIdSupplier.get());
         }
 
         final Producer<byte[], byte[]> producer = new KafkaProducer<>(properties);
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtilsTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtilsTest.java
new file mode 100644
index 0000000..1a20d2d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtilsTest.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.kafka.pubsub;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.function.Supplier;
+
+import org.junit.Test;
+
+public class KafkaProcessorUtilsTest {
+
+  @Test
+  public void getTransactionalIdSupplierWithPrefix() {
+    Supplier<String> prefix = KafkaProcessorUtils.getTransactionalIdSupplier("prefix");
+    String id = prefix.get();
+    assertTrue(id.startsWith("prefix"));
+    assertEquals(42, id.length());
+  }
+
+  @Test
+  public void getTransactionalIdSupplierWithEmptyPrefix() {
+    Supplier<String> prefix = KafkaProcessorUtils.getTransactionalIdSupplier(null);
+    assertEquals(36, prefix.get().length() );
+  }
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java
index afaf841..4deeba3 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java
@@ -28,7 +28,6 @@ import org.apache.nifi.logging.ComponentLog;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-
 public class TestPublisherPool {
 
     @Test
@@ -38,7 +37,7 @@ public class TestPublisherPool {
         kafkaProperties.put("key.serializer", ByteArraySerializer.class.getName());
         kafkaProperties.put("value.serializer", ByteArraySerializer.class.getName());
 
-        final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L, false, null, StandardCharsets.UTF_8);
+        final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L, false, null, null, StandardCharsets.UTF_8);
         assertEquals(0, pool.available());
 
         final PublisherLease lease = pool.obtainPublisher();
@@ -55,7 +54,7 @@ public class TestPublisherPool {
         kafkaProperties.put("key.serializer", ByteArraySerializer.class.getName());
         kafkaProperties.put("value.serializer", ByteArraySerializer.class.getName());
 
-        final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L, false, null, StandardCharsets.UTF_8);
+        final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L, false, null, null, StandardCharsets.UTF_8);
         assertEquals(0, pool.available());
 
         final PublisherLease lease = pool.obtainPublisher();
@@ -65,5 +64,4 @@ public class TestPublisherPool {
         lease.close();
         assertEquals(0, pool.available());
     }
-
 }
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
index 3cde7ce..7c3c578 100644
--- 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
@@ -25,7 +25,9 @@ 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;
 
 import org.apache.commons.lang3.StringUtils;
@@ -332,6 +334,16 @@ final class KafkaProcessorUtils {
     }
 
     /**
+     * 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 />
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 9182084..414523b 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
@@ -62,6 +62,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 
 @Tags({"Apache", "Kafka", "Record", "csv", "json", "avro", "logs", "Put", "Send", "Message", "PubSub", "2.0"})
@@ -220,6 +221,14 @@ public class PublishKafkaRecord_2_0 extends AbstractProcessor {
         .defaultValue("true")
         .required(true)
         .build();
+    static final PropertyDescriptor TRANSACTIONAL_ID_PREFIX = new PropertyDescriptor.Builder()
+        .name("transactional-id-prefix")
+        .displayName("Transactional Id Prefix")
+        .description("When Use Transaction is set to true, KafkaProducer config 'transactional.id' will be a generated UUID and will be prefixed with this string.")
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+        .required(false)
+        .build();
     static final PropertyDescriptor MESSAGE_HEADER_ENCODING = new PropertyDescriptor.Builder()
         .name("message-header-encoding")
         .displayName("Message Header Encoding")
@@ -252,6 +261,7 @@ public class PublishKafkaRecord_2_0 extends AbstractProcessor {
         properties.add(RECORD_READER);
         properties.add(RECORD_WRITER);
         properties.add(USE_TRANSACTIONS);
+        properties.add(TRANSACTIONAL_ID_PREFIX);
         properties.add(DELIVERY_GUARANTEE);
         properties.add(ATTRIBUTE_NAME_REGEX);
         properties.add(MESSAGE_HEADER_ENCODING);
@@ -334,6 +344,8 @@ public class PublishKafkaRecord_2_0 extends AbstractProcessor {
         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);
 
         final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
         final Charset charset = Charset.forName(charsetName);
@@ -344,7 +356,7 @@ public class PublishKafkaRecord_2_0 extends AbstractProcessor {
         kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize));
 
-        return new PublisherPool(kafkaProperties, getLogger(), maxMessageSize, maxAckWaitMillis, useTransactions, attributeNamePattern, charset);
+        return new PublisherPool(kafkaProperties, getLogger(), maxMessageSize, maxAckWaitMillis, useTransactions, transactionalIdSupplier, attributeNamePattern, charset);
     }
 
     @OnStopped
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 251c10e..fdfc2e8 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
@@ -56,6 +56,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 
 @Tags({"Apache", "Kafka", "Put", "Send", "Message", "PubSub", "2.0"})
@@ -222,6 +223,14 @@ public class PublishKafka_2_0 extends AbstractProcessor {
         .defaultValue("true")
         .required(true)
         .build();
+    static final PropertyDescriptor TRANSACTIONAL_ID_PREFIX = new PropertyDescriptor.Builder()
+        .name("transactional-id-prefix")
+        .displayName("Transactional Id Prefix")
+        .description("When Use Transaction is set to true, KafkaProducer config 'transactional.id' will be a generated UUID and will be prefixed with this string.")
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+        .required(false)
+        .build();
     static final PropertyDescriptor MESSAGE_HEADER_ENCODING = new PropertyDescriptor.Builder()
         .name("message-header-encoding")
         .displayName("Message Header Encoding")
@@ -253,6 +262,7 @@ public class PublishKafka_2_0 extends AbstractProcessor {
         properties.add(TOPIC);
         properties.add(DELIVERY_GUARANTEE);
         properties.add(USE_TRANSACTIONS);
+        properties.add(TRANSACTIONAL_ID_PREFIX);
         properties.add(ATTRIBUTE_NAME_REGEX);
         properties.add(MESSAGE_HEADER_ENCODING);
         properties.add(KEY);
@@ -330,6 +340,8 @@ public class PublishKafka_2_0 extends AbstractProcessor {
         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);
 
         final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
         final Charset charset = Charset.forName(charsetName);
@@ -340,7 +352,7 @@ public class PublishKafka_2_0 extends AbstractProcessor {
         kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize));
 
-        return new PublisherPool(kafkaProperties, getLogger(), maxMessageSize, maxAckWaitMillis, useTransactions, attributeNamePattern, charset);
+        return new PublisherPool(kafkaProperties, getLogger(), maxMessageSize, maxAckWaitMillis, useTransactions, transactionalIdSupplier, attributeNamePattern, charset);
     }
 
     @OnStopped
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
index d5caa8d..ec07da9 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
@@ -21,9 +21,9 @@ import java.io.Closeable;
 import java.nio.charset.Charset;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.UUID;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 
 import org.apache.kafka.clients.producer.KafkaProducer;
@@ -39,11 +39,12 @@ public class PublisherPool implements Closeable {
     private final boolean useTransactions;
     private final Pattern attributeNameRegex;
     private final Charset headerCharacterSet;
+    private Supplier<String> transactionalIdSupplier;
 
     private volatile boolean closed = false;
 
     PublisherPool(final Map<String, Object> kafkaProperties, final ComponentLog logger, final int maxMessageSize, final long maxAckWaitMillis,
-        final boolean useTransactions, final Pattern attributeNameRegex, final Charset headerCharacterSet) {
+        final boolean useTransactions, final Supplier<String> transactionalIdSupplier, final Pattern attributeNameRegex, final Charset headerCharacterSet) {
         this.logger = logger;
         this.publisherQueue = new LinkedBlockingQueue<>();
         this.kafkaProperties = kafkaProperties;
@@ -52,6 +53,7 @@ public class PublisherPool implements Closeable {
         this.useTransactions = useTransactions;
         this.attributeNameRegex = attributeNameRegex;
         this.headerCharacterSet = headerCharacterSet;
+        this.transactionalIdSupplier = transactionalIdSupplier;
     }
 
     public PublisherLease obtainPublisher() {
@@ -71,7 +73,7 @@ public class PublisherPool implements Closeable {
     private PublisherLease createLease() {
         final Map<String, Object> properties = new HashMap<>(kafkaProperties);
         if (useTransactions) {
-            properties.put("transactional.id", UUID.randomUUID().toString());
+            properties.put("transactional.id", transactionalIdSupplier.get());
         }
 
         final Producer<byte[], byte[]> producer = new KafkaProducer<>(properties);
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtilsTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtilsTest.java
new file mode 100644
index 0000000..1a20d2d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtilsTest.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.kafka.pubsub;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.function.Supplier;
+
+import org.junit.Test;
+
+public class KafkaProcessorUtilsTest {
+
+  @Test
+  public void getTransactionalIdSupplierWithPrefix() {
+    Supplier<String> prefix = KafkaProcessorUtils.getTransactionalIdSupplier("prefix");
+    String id = prefix.get();
+    assertTrue(id.startsWith("prefix"));
+    assertEquals(42, id.length());
+  }
+
+  @Test
+  public void getTransactionalIdSupplierWithEmptyPrefix() {
+    Supplier<String> prefix = KafkaProcessorUtils.getTransactionalIdSupplier(null);
+    assertEquals(36, prefix.get().length() );
+  }
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java
index 070dea1..275d867 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java
@@ -37,7 +37,7 @@ public class TestPublisherPool {
         kafkaProperties.put("key.serializer", ByteArraySerializer.class.getName());
         kafkaProperties.put("value.serializer", ByteArraySerializer.class.getName());
 
-        final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L, false, null, StandardCharsets.UTF_8);
+        final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L, false, null, null, StandardCharsets.UTF_8);
         assertEquals(0, pool.available());
 
         final PublisherLease lease = pool.obtainPublisher();
@@ -54,7 +54,7 @@ public class TestPublisherPool {
         kafkaProperties.put("key.serializer", ByteArraySerializer.class.getName());
         kafkaProperties.put("value.serializer", ByteArraySerializer.class.getName());
 
-        final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L, false, null, StandardCharsets.UTF_8);
+        final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L, false, null, null, StandardCharsets.UTF_8);
         assertEquals(0, pool.available());
 
         final PublisherLease lease = pool.obtainPublisher();
@@ -64,5 +64,4 @@ public class TestPublisherPool {
         lease.close();
         assertEquals(0, pool.available());
     }
-
 }