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 ae44837c1f..de3ff83cae 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 @@
org.apache.nifinifi-kerberos-credentials-service-api
+
+ org.apache.nifi
+ nifi-kerberos-user-service-api
+
+
+ org.apache.nifi
+ nifi-security-kerberos
+ 1.19.0-SNAPSHOT
+
+
+ org.apache.nifi
+ nifi-kafka-shared
+ org.apache.kafkakafka-clients
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 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 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 props = new HashMap<>();
- KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props);
+ final KafkaPropertyProvider kafkaPropertyProvider = new StandardKafkaPropertyProvider(ConsumerConfig.class);
+ final Map 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 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 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 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 props = new HashMap<>();
- KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props);
+ final KafkaPropertyProvider kafkaPropertyProvider = new StandardKafkaPropertyProvider(ConsumerConfig.class);
+ final Map 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 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 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 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 records) {
- records.partitions().stream().forEach(partition -> {
+ records.partitions().forEach(partition -> {
List> 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 attributes = new HashMap<>();
- attributes.putAll(writeResult.getAttributes());
+ final Map 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 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 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 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 :")
- .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 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 getCommonPropertyDescriptors() {
- return Arrays.asList(
- BOOTSTRAP_SERVERS,
- SECURITY_PROTOCOL,
- JAAS_SERVICE_NAME,
- KERBEROS_CREDENTIALS_SERVICE,
- USER_PRINCIPAL,
- USER_KEYTAB,
- SSL_CONTEXT_SERVICE
- );
- }
-
- public static Collection validateCommonProperties(final ValidationContext validationContext) {
- List 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 <security.protocol>://<bootstrap.servers>/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 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 getTransactionalIdSupplier(String prefix) {
- return () -> (prefix == null ? "" : prefix) + UUID.randomUUID().toString();
- }
-
- /**
- * Method used to configure the 'sasl.jaas.config' property based on KAFKA-4259
- * https://cwiki.apache.org/confluence/display/KAFKA/KIP-85%3A+Dynamic+JAAS+configuration+for+Kafka+clients
- *
- * It expects something with the following format:
- *
- * <LoginModuleClass> <ControlFlag> *(<OptionName>=<OptionValue>);
- * ControlFlag = required / requisite / sufficient / optional
- *
- * @param mapToPopulate Map of configuration properties
- * @param context Context
- */
- private static void setJaasConfig(Map 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 getPublicStaticStringFieldValues(final Class>... classes) {
- final Set 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 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 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 customValidate(final ValidationContext validationContext) {
- final List results = new ArrayList<>();
- results.addAll(KafkaProcessorUtils.validateCommonProperties(validationContext));
+ final List 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 transactionalIdSupplier = KafkaProcessorUtils.getTransactionalIdSupplier(transactionalIdPrefix);
+ Supplier transactionalIdSupplier = new TransactionIdSupplier(transactionalIdPrefix);
final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
final Charset charset = Charset.forName(charsetName);
- final Map kafkaProperties = new HashMap<>();
- KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
+ final KafkaPropertyProvider propertyProvider = new StandardKafkaPropertyProvider(ProducerConfig.class);
+ final Map 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 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();
+ 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);
- }
+ 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 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 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 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 customValidate(final ValidationContext validationContext) {
- final List results = new ArrayList<>();
- results.addAll(KafkaProcessorUtils.validateCommonProperties(validationContext));
+ final List 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 transactionalIdSupplier = KafkaProcessorUtils.getTransactionalIdSupplier(transactionalIdPrefix);
+ Supplier transactionalIdSupplier = new TransactionIdSupplier(transactionalIdPrefix);
final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
final Charset charset = Charset.forName(charsetName);
- final Map kafkaProperties = new HashMap<>();
- KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
+ final KafkaPropertyProvider propertyProvider = new StandardKafkaPropertyProvider(ProducerConfig.class);
+ final Map 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 producer;
@Override
- protected void init(final ControllerServiceInitializationContext context) throws InitializationException {
+ protected void init(final ControllerServiceInitializationContext context) {
final List 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 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 kafkaProperties = new HashMap<>();
- buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
+ final KafkaPropertyProvider kafkaPropertyProvider = new StandardKafkaPropertyProvider(ConsumerConfig.class);
+ final Map 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 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
- * https://cwiki.apache.org/confluence/display/KAFKA/KIP-85%3A+Dynamic+JAAS+configuration+for+Kafka+clients
- *
- * It expects something with the following format:
- *
- * <LoginModuleClass> <ControlFlag> *(<OptionName>=<OptionValue>);
- * ControlFlag = required / requisite / sufficient / optional
- *
- * @param mapToPopulate Map of configuration properties
- * @param context Context
- */
- private static void setJaasConfig(Map 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(KafkaProcessorUtils.USER_PRINCIPAL, "nifi@APACHE.COM");
+ runner.setProperty(ConsumeKafka_1_0.KERBEROS_SERVICE_NAME, "kafka");
runner.assertNotValid();
- runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "not.A.File");
+ runner.setProperty(ConsumeKafka_1_0.KERBEROS_PRINCIPAL, "nifi@APACHE.COM");
runner.assertNotValid();
- runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "src/test/resources/server.properties");
+ runner.setProperty(ConsumeKafka_1_0.KERBEROS_KEYTAB, "not.A.File");
+ runner.assertNotValid();
+
+ 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 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(KafkaProcessorUtils.USER_PRINCIPAL, "nifi@APACHE.COM");
+ runner.setProperty(ConsumeKafkaRecord_1_0.KERBEROS_SERVICE_NAME, "kafka");
runner.assertNotValid();
- runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "not.A.File");
+ runner.setProperty(ConsumeKafkaRecord_1_0.KERBEROS_PRINCIPAL, "nifi@APACHE.COM");
runner.assertNotValid();
- runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "src/test/resources/server.properties");
+ runner.setProperty(ConsumeKafkaRecord_1_0.KERBEROS_KEYTAB, "not.A.File");
+ runner.assertNotValid();
+
+ 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 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 116e483f27..644af77411 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 @@
org.apache.nifinifi-kerberos-credentials-service-api
-
+
+ org.apache.nifi
+ nifi-kerberos-user-service-api
+
+
+ org.apache.nifi
+ nifi-security-kerberos
+ 1.19.0-SNAPSHOT
+
+
+ org.apache.nifi
+ nifi-kafka-shared
+ org.apache.kafkakafka-clients
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 property is set to true, FlowFiles that are emitted have an attribute named '" + KafkaProcessorUtils.KAFKA_KEY +
+ .description("If the 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 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 customValidate(final ValidationContext validationContext) {
- final Collection validationResults = KafkaProcessorUtils.validateCommonProperties(validationContext);
+ final Collection 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 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 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 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 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 customValidate(final ValidationContext validationContext) {
- final Collection validationResults = KafkaProcessorUtils.validateCommonProperties(validationContext);
+ final Collection 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 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 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 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 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 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 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 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 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 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 :")
- .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 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 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 validateCommonProperties(final ValidationContext validationContext) {
- List 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 <security.protocol>://<bootstrap.servers>/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 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 getTransactionalIdSupplier(String prefix) {
- return () -> (prefix == null ? "" : prefix) + UUID.randomUUID().toString();
- }
-
- /**
- * Method used to configure the 'sasl.jaas.config' property based on KAFKA-4259
- * https://cwiki.apache.org/confluence/display/KAFKA/KIP-85%3A+Dynamic+JAAS+configuration+for+Kafka+clients
- *
- * It expects something with the following format:
- *
- * <LoginModuleClass> <ControlFlag> *(<OptionName>=<OptionValue>);
- * ControlFlag = required / requisite / sufficient / optional
- *
- * @param mapToPopulate Map of configuration properties
- * @param context Context
- */
- private static void setJaasConfig(Map 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 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 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 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 getPublicStaticStringFieldValues(final Class>... classes) {
- final Set 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 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 customValidate(final ValidationContext validationContext) {
- final List results = new ArrayList<>();
- results.addAll(KafkaProcessorUtils.validateCommonProperties(validationContext));
+ final List 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 transactionalIdSupplier = KafkaProcessorUtils.getTransactionalIdSupplier(transactionalIdPrefix);
+ Supplier transactionalIdSupplier = new TransactionIdSupplier(transactionalIdPrefix);
final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
final Charset charset = Charset.forName(charsetName);
- final Map kafkaProperties = new HashMap<>();
- KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
+ final KafkaPropertyProvider propertyProvider = new StandardKafkaPropertyProvider(ProducerConfig.class);
+ final Map 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 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();
+ 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);
- }
+ 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 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 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 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 customValidate(final ValidationContext validationContext) {
- final List results = new ArrayList<>();
- results.addAll(KafkaProcessorUtils.validateCommonProperties(validationContext));
+ final List 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 transactionalIdSupplier = KafkaProcessorUtils.getTransactionalIdSupplier(transactionalIdPrefix);
+ Supplier transactionalIdSupplier = new TransactionIdSupplier(transactionalIdPrefix);
final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
final Charset charset = Charset.forName(charsetName);
- final Map kafkaProperties = new HashMap<>();
- KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
+ final KafkaPropertyProvider propertyProvider = new StandardKafkaPropertyProvider(ProducerConfig.class);
+ final Map 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 producer;
@Override
- protected void init(final ControllerServiceInitializationContext context) throws InitializationException {
+ protected void init(final ControllerServiceInitializationContext context) {
final List 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 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 kafkaProperties = new HashMap<>();
- buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
+ final KafkaPropertyProvider propertyProvider = new StandardKafkaPropertyProvider(ProducerConfig.class);
+ final Map 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 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
- * https://cwiki.apache.org/confluence/display/KAFKA/KIP-85%3A+Dynamic+JAAS+configuration+for+Kafka+clients
- *
- * It expects something with the following format:
- *
- * <LoginModuleClass> <ControlFlag> *(<OptionName>=<OptionValue>);
- * ControlFlag = required / requisite / sufficient / optional
- *
- * @param mapToPopulate Map of configuration properties
- * @param context Context
- */
- private static void setJaasConfig(Map 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(KafkaProcessorUtils.USER_PRINCIPAL, "nifi@APACHE.COM");
+ runner.setProperty(ConsumeKafkaRecord_2_0.KERBEROS_SERVICE_NAME, "kafka");
runner.assertNotValid();
- runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "not.A.File");
+ runner.setProperty(ConsumeKafkaRecord_2_0.KERBEROS_PRINCIPAL, "nifi@APACHE.COM");
runner.assertNotValid();
- runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "src/test/resources/server.properties");
+ runner.setProperty(ConsumeKafkaRecord_2_0.KERBEROS_KEYTAB, "not.A.File");
+ runner.assertNotValid();
+
+ 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(KafkaProcessorUtils.USER_PRINCIPAL, "nifi@APACHE.COM");
+ runner.setProperty(ConsumeKafka_2_0.KERBEROS_SERVICE_NAME, "kafka");
runner.assertNotValid();
- runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "not.A.File");
+ runner.setProperty(ConsumeKafka_2_0.KERBEROS_PRINCIPAL, "nifi@APACHE.COM");
runner.assertNotValid();
- runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "src/test/resources/server.properties");
+ runner.setProperty(ConsumeKafka_2_0.KERBEROS_KEYTAB, "not.A.File");
+ runner.assertNotValid();
+
+ 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