Skip to content

Commit

Permalink
NIFI-6218 Support setting transactional.id in PublishKafka/PublishKaf…
Browse files Browse the repository at this point in the history
…kaRecord

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 <ijokarumawak@apache.org>
  • Loading branch information
szaboferee authored and ijokarumawak committed Jun 7, 2019
1 parent 2d3fce8 commit aefdc93
Show file tree
Hide file tree
Showing 18 changed files with 258 additions and 26 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,9 @@
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;
Expand Down Expand Up @@ -331,6 +333,16 @@ static void buildCommonKafkaProperties(final ProcessContext context, final Class
}
}

/**
* 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 />
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@
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"})
Expand Down Expand Up @@ -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")
Expand Down Expand Up @@ -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);
Expand Down Expand Up @@ -332,6 +342,8 @@ protected PublisherPool createPublisherPool(final ProcessContext context) {
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);
Expand All @@ -342,7 +354,7 @@ protected PublisherPool createPublisherPool(final ProcessContext context) {
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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
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;
Expand Down Expand Up @@ -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")
Expand Down Expand Up @@ -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);
Expand Down Expand Up @@ -329,6 +339,8 @@ protected PublisherPool createPublisherPool(final ProcessContext context) {
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);
Expand All @@ -339,7 +351,7 @@ protected PublisherPool createPublisherPool(final ProcessContext context) {
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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,9 @@
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;
Expand All @@ -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;
Expand All @@ -52,6 +53,7 @@ public class PublisherPool implements Closeable {
this.useTransactions = useTransactions;
this.attributeNameRegex = attributeNameRegex;
this.headerCharacterSet = headerCharacterSet;
this.transactionalIdSupplier = transactionalIdSupplier;
}

public PublisherLease obtainPublisher() {
Expand All @@ -71,7 +73,7 @@ public PublisherLease obtainPublisher() {
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);
Expand Down
Original file line number Diff line number Diff line change
@@ -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() );
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@
import org.junit.Test;
import org.mockito.Mockito;


public class TestPublisherPool {

@Test
Expand All @@ -38,7 +37,7 @@ public void testLeaseCloseReturnsToPool() {
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();
Expand All @@ -55,7 +54,7 @@ public void testPoisonedLeaseNotReturnedToPool() {
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();
Expand All @@ -65,5 +64,4 @@ public void testPoisonedLeaseNotReturnedToPool() {
lease.close();
assertEquals(0, pool.available());
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,9 @@
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;
Expand Down Expand Up @@ -331,6 +333,16 @@ static void buildCommonKafkaProperties(final ProcessContext context, final Class
}
}

/**
* 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 />
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@
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"})
Expand Down Expand Up @@ -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")
Expand Down Expand Up @@ -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);
Expand Down Expand Up @@ -334,6 +344,8 @@ protected PublisherPool createPublisherPool(final ProcessContext context) {
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);
Expand All @@ -344,7 +356,7 @@ protected PublisherPool createPublisherPool(final ProcessContext context) {
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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
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;
Expand Down Expand Up @@ -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")
Expand Down Expand Up @@ -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);
Expand Down Expand Up @@ -331,6 +341,8 @@ protected PublisherPool createPublisherPool(final ProcessContext context) {
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);
Expand All @@ -341,7 +353,7 @@ protected PublisherPool createPublisherPool(final ProcessContext context) {
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
Expand Down
Loading

0 comments on commit aefdc93

Please sign in to comment.