NIFI-10819 Restricted JndiLoginModule for Kafka JAAS Config

- Refactored property configuration and validation to nifi-kafka-shared

Signed-off-by: Joe Gresock <jgresock@gmail.com>

This closes #6690.
This commit is contained in:
exceptionfactory 2022-11-15 14:08:24 -06:00 committed by Joe Gresock
parent 09bc5bcb5a
commit 8e417c890a
No known key found for this signature in database
GPG Key ID: 37F5B9B6E258C8B7
56 changed files with 2255 additions and 1226 deletions

View File

@ -61,6 +61,10 @@
<artifactId>nifi-security-kerberos</artifactId>
<version>1.19.0-SNAPSHOT</version>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-kafka-shared</artifactId>
</dependency>
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-clients</artifactId>

View File

@ -36,6 +36,15 @@ import org.apache.nifi.components.PropertyDescriptor.Builder;
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.KeyEncoding;
import org.apache.nifi.kafka.shared.property.OutputStrategy;
import org.apache.nifi.kafka.shared.validation.KafkaClientCustomValidationFunction;
import org.apache.nifi.kafka.shared.property.KeyFormat;
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.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
@ -52,7 +61,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;
@ -60,14 +68,9 @@ import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
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.KEY_AS_BYTE_ARRAY;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.KEY_AS_RECORD;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.KEY_AS_STRING;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.OUTPUT_USE_VALUE;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.OUTPUT_USE_WRAPPER;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_PARTITION;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_TIMESTAMP;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_TOPIC;
@CapabilityDescription("Consumes messages from Apache Kafka specifically built against the Kafka 2.6 Consumer API. "
+ "The complementary NiFi processor for sending messages is PublishKafkaRecord_2_6. Please note that, at this time, the Processor assumes that "
@ -80,9 +83,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 = KAFKA_PARTITION, description = "The partition of the topic the records are from"),
@WritesAttribute(attribute = KAFKA_TIMESTAMP, description = "The timestamp of the message in the partition of the topic."),
@WritesAttribute(attribute = 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.",
@ -91,7 +94,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 = ExpressionLanguageScope.VARIABLE_REGISTRY)
@SeeAlso({ConsumeKafka_2_6.class, PublishKafka_2_6.class, PublishKafkaRecord_2_6.class})
public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements VerifiableProcessor {
public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements KafkaClientComponent, VerifiableProcessor {
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");
@ -222,17 +225,17 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
.displayName("Output Strategy")
.description("The format used to output the Kafka record into a FlowFile record.")
.required(true)
.defaultValue(OUTPUT_USE_VALUE.getValue())
.allowableValues(OUTPUT_USE_VALUE, OUTPUT_USE_WRAPPER)
.defaultValue(OutputStrategy.USE_VALUE.getValue())
.allowableValues(OutputStrategy.class)
.build();
static final PropertyDescriptor KEY_FORMAT = new PropertyDescriptor.Builder()
.name("key-format")
.displayName("Key Format")
.description("Specifies how to represent the Kafka Record's Key in the output")
.required(true)
.defaultValue(KEY_AS_BYTE_ARRAY.getValue())
.allowableValues(KEY_AS_STRING, KEY_AS_BYTE_ARRAY, KEY_AS_RECORD)
.dependsOn(OUTPUT_STRATEGY, OUTPUT_USE_WRAPPER)
.defaultValue(KeyFormat.BYTE_ARRAY.getValue())
.allowableValues(KeyFormat.class)
.dependsOn(OUTPUT_STRATEGY, OutputStrategy.USE_WRAPPER.getValue())
.build();
static final PropertyDescriptor KEY_RECORD_READER = new PropertyDescriptor.Builder()
.name("key-record-reader")
@ -240,7 +243,7 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
.description("The Record Reader to use for parsing the Kafka Record's key into a Record")
.identifiesControllerService(RecordReaderFactory.class)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.dependsOn(KEY_FORMAT, KEY_AS_RECORD)
.dependsOn(KEY_FORMAT, KeyFormat.RECORD.getValue())
.build();
static final PropertyDescriptor HEADER_NAME_REGEX = new Builder()
.name("header-name-regex")
@ -254,7 +257,7 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
.addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(false)
.dependsOn(OUTPUT_STRATEGY, OUTPUT_USE_VALUE)
.dependsOn(OUTPUT_STRATEGY, OutputStrategy.USE_VALUE.getValue())
.build();
static final PropertyDescriptor SEPARATE_BY_KEY = new Builder()
@ -268,12 +271,12 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
static final PropertyDescriptor KEY_ATTRIBUTE_ENCODING = new PropertyDescriptor.Builder()
.name("key-attribute-encoding")
.displayName("Key Attribute Encoding")
.description("If the <Separate By Key> property is set to true, FlowFiles that are emitted have an attribute named '" + KafkaProcessorUtils.KAFKA_KEY +
.description("If the <Separate By Key> property is set to true, FlowFiles that are emitted have an attribute named '" + KafkaFlowFileAttribute.KAFKA_KEY +
"'. This property dictates how the value of the attribute should be encoded.")
.required(true)
.defaultValue(UTF8_ENCODING.getValue())
.allowableValues(UTF8_ENCODING, HEX_ENCODING, DO_NOT_ADD_KEY_AS_ATTRIBUTE)
.dependsOn(OUTPUT_STRATEGY, OUTPUT_USE_VALUE)
.defaultValue(KeyEncoding.UTF8.getValue())
.allowableValues(KeyEncoding.class)
.dependsOn(OUTPUT_STRATEGY, OutputStrategy.USE_VALUE.getValue())
.build();
static final Relationship REL_SUCCESS = new Relationship.Builder()
@ -294,7 +297,7 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
static {
final List<PropertyDescriptor> descriptors = new ArrayList<>();
descriptors.add(KafkaProcessorUtils.BOOTSTRAP_SERVERS);
descriptors.add(BOOTSTRAP_SERVERS);
descriptors.add(TOPICS);
descriptors.add(TOPIC_TYPE);
descriptors.add(RECORD_READER);
@ -308,16 +311,16 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
descriptors.add(COMMIT_OFFSETS);
descriptors.add(MAX_UNCOMMITTED_TIME);
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(SEPARATE_BY_KEY);
descriptors.add(AUTO_OFFSET_RESET);
descriptors.add(MESSAGE_HEADER_ENCODING);
@ -356,7 +359,7 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
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(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
@ -364,7 +367,8 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
@Override
protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
final Collection<ValidationResult> validationResults = KafkaProcessorUtils.validateCommonProperties(validationContext);
final KafkaClientCustomValidationFunction validationFunction = new KafkaClientCustomValidationFunction();
final Collection<ValidationResult> validationResults = validationFunction.apply(validationContext);
final ValidationResult consumerPartitionsResult = ConsumerPartitionsUtil.validateConsumePartitions(validationContext.getAllProperties());
validationResults.add(consumerPartitionsResult);
@ -419,16 +423,16 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
final Long maxUncommittedTime = context.getProperty(MAX_UNCOMMITTED_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
final boolean commitOffsets = context.getProperty(COMMIT_OFFSETS).asBoolean();
final Map<String, Object> props = new HashMap<>();
KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props);
final KafkaPropertyProvider propertyProvider = new StandardKafkaPropertyProvider(ConsumerConfig.class);
final Map<String, Object> props = propertyProvider.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_2_6.TOPICS).evaluateAttributeExpressions().getValue();
final String topicType = context.getProperty(ConsumeKafkaRecord_2_6.TOPIC_TYPE).evaluateAttributeExpressions().getValue();
final List<String> topics = new ArrayList<>();
final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue();
final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
final String securityProtocol = context.getProperty(SECURITY_PROTOCOL).getValue();
final String bootstrapServers = context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
@ -474,7 +478,7 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
bootstrapServers, log, honorTransactions, charset, headerNamePattern, separateByKey, keyEncoding, partitionsToConsume,
commitOffsets, outputStrategy, keyFormat, keyReaderFactory);
} else {
getLogger().error("Subscription type has an unknown value {}", new Object[] {topicType});
getLogger().error("Subscription type has an unknown value {}", topicType);
return null;
}
}
@ -499,12 +503,12 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
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();
@ -535,7 +539,7 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
}
} 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);

View File

@ -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.property.KeyEncoding;
import org.apache.nifi.kafka.shared.validation.KafkaClientCustomValidationFunction;
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.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.components.ConfigVerificationResult;
@ -50,7 +57,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;
@ -58,20 +64,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.6 Consumer API. "
+ "The complementary NiFi processor for sending messages is PublishKafka_2_6.")
@Tags({"Kafka", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume", "2.6"})
@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.",
@ -79,7 +82,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_6 extends AbstractProcessor implements VerifiableProcessor {
public class ConsumeKafka_2_6 extends AbstractProcessor implements KafkaClientComponent, VerifiableProcessor {
static final AllowableValue OFFSET_EARLIEST = new AllowableValue("earliest", "earliest", "Automatically reset the offset to the earliest offset");
@ -131,10 +134,10 @@ public class ConsumeKafka_2_6 extends AbstractProcessor implements VerifiablePro
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()
@ -252,7 +255,7 @@ public class ConsumeKafka_2_6 extends AbstractProcessor implements VerifiablePro
static {
final List<PropertyDescriptor> descriptors = new ArrayList<>();
descriptors.add(KafkaProcessorUtils.BOOTSTRAP_SERVERS);
descriptors.add(BOOTSTRAP_SERVERS);
descriptors.add(TOPICS);
descriptors.add(TOPIC_TYPE);
descriptors.add(GROUP_ID);
@ -261,17 +264,17 @@ public class ConsumeKafka_2_6 extends AbstractProcessor implements VerifiablePro
descriptors.add(HONOR_TRANSACTIONS);
descriptors.add(MESSAGE_DEMARCATOR);
descriptors.add(SEPARATE_BY_KEY);
descriptors.add(KafkaProcessorUtils.SECURITY_PROTOCOL);
descriptors.add(KafkaProcessorUtils.SASL_MECHANISM);
descriptors.add(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE);
descriptors.add(KafkaProcessorUtils.SELF_CONTAINED_KERBEROS_USER_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(SELF_CONTAINED_KERBEROS_USER_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(KEY_ATTRIBUTE_ENCODING);
descriptors.add(AUTO_OFFSET_RESET);
descriptors.add(MESSAGE_HEADER_ENCODING);
@ -307,7 +310,7 @@ public class ConsumeKafka_2_6 extends AbstractProcessor implements VerifiablePro
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();
@ -315,7 +318,8 @@ public class ConsumeKafka_2_6 extends AbstractProcessor implements VerifiablePro
@Override
protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
final Collection<ValidationResult> validationResults = KafkaProcessorUtils.validateCommonProperties(validationContext);
final KafkaClientCustomValidationFunction validationFunction = new KafkaClientCustomValidationFunction();
final Collection<ValidationResult> validationResults = validationFunction.apply(validationContext);
final ValidationResult consumerPartitionsResult = ConsumerPartitionsUtil.validateConsumePartitions(validationContext.getAllProperties());
validationResults.add(consumerPartitionsResult);
@ -372,8 +376,8 @@ public class ConsumeKafka_2_6 extends AbstractProcessor implements VerifiablePro
final byte[] demarcator = context.getProperty(ConsumeKafka_2_6.MESSAGE_DEMARCATOR).isSet()
? context.getProperty(ConsumeKafka_2_6.MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8) : null;
final Map<String, Object> props = new HashMap<>();
KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props);
final KafkaPropertyProvider propertyProvider = new StandardKafkaPropertyProvider(ConsumerConfig.class);
final Map<String, Object> props = propertyProvider.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());
@ -382,8 +386,8 @@ public class ConsumeKafka_2_6 extends AbstractProcessor implements VerifiablePro
final String topicType = context.getProperty(ConsumeKafka_2_6.TOPIC_TYPE).evaluateAttributeExpressions().getValue();
final List<String> topics = new ArrayList<>();
final String keyEncoding = context.getProperty(KEY_ATTRIBUTE_ENCODING).getValue();
final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue();
final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
final String securityProtocol = context.getProperty(SECURITY_PROTOCOL).getValue();
final String bootstrapServers = context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
final boolean honorTransactions = context.getProperty(HONOR_TRANSACTIONS).asBoolean();
final int commsTimeoutMillis = context.getProperty(COMMS_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue();
props.put(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG, commsTimeoutMillis);
@ -418,7 +422,7 @@ public class ConsumeKafka_2_6 extends AbstractProcessor implements VerifiablePro
return new ConsumerPool(maxLeases, demarcator, separateByKey, props, topicPattern, maxUncommittedTime, keyEncoding, securityProtocol,
bootstrapServers, log, honorTransactions, charset, headerNamePattern, partitionsToConsume, commitOffsets);
} else {
getLogger().error("Subscription type has an unknown value {}", new Object[] {topicType});
getLogger().error("Subscription type has an unknown value {}", topicType);
return null;
}
}
@ -443,12 +447,12 @@ public class ConsumeKafka_2_6 extends AbstractProcessor implements VerifiablePro
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();
@ -479,7 +483,7 @@ public class ConsumeKafka_2_6 extends AbstractProcessor implements VerifiablePro
}
} 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);

View File

@ -28,6 +28,11 @@ 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.kafka.shared.property.KeyFormat;
import org.apache.nifi.kafka.shared.property.OutputStrategy;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.exception.ProcessException;
@ -69,10 +74,17 @@ import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_CONSUMER_GROUP_ID;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_CONSUMER_OFFSETS_COMMITTED;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_COUNT;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_LEADER_EPOCH;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_MAX_OFFSET;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_OFFSET;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_PARTITION;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_TIMESTAMP;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_TOPIC;
import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_2_6.REL_PARSE_FAILURE;
import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_2_6.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
@ -170,7 +182,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
*/
@Override
public void onPartitionsRevoked(final Collection<TopicPartition> partitions) {
logger.debug("Rebalance Alert: Partitions '{}' revoked for lease '{}' with consumer '{}'", new Object[]{partitions, this, kafkaConsumer});
logger.debug("Rebalance Alert: Partitions '{}' revoked for lease '{}' with consumer '{}'", partitions, this, kafkaConsumer);
commit();
}
@ -183,7 +195,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
*/
@Override
public void onPartitionsAssigned(final Collection<TopicPartition> partitions) {
logger.debug("Rebalance Alert: Partitions '{}' assigned for lease '{}' with consumer '{}'", new Object[]{partitions, this, kafkaConsumer});
logger.debug("Rebalance Alert: Partitions '{}' assigned for lease '{}' with consumer '{}'", partitions, this, kafkaConsumer);
}
public List<TopicPartition> getAssignedPartitions() {
@ -231,7 +243,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
*
@ -262,7 +273,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
for (final FlowFile flowFile : bundledFlowFiles) {
final String recordCountAttribute = flowFile.getAttribute("record.count");
final String recordCount = recordCountAttribute == null ? "1" : recordCountAttribute;
logger.debug("Transferred {} with {} records, max offset of {}", flowFile, recordCount, flowFile.getAttribute(KafkaProcessorUtils.KAFKA_MAX_OFFSET));
logger.debug("Transferred {} with {} records, max offset of {}", flowFile, recordCount, flowFile.getAttribute(KAFKA_MAX_OFFSET));
}
}
}
@ -410,9 +421,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
@ -446,8 +457,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
return false;
}
final Map<String, String> attributes = new HashMap<>();
attributes.putAll(writeResult.getAttributes());
final Map<String, String> attributes = new HashMap<>(writeResult.getAttributes());
attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType());
bundle.flowFile = getProcessSession().putAllAttributes(bundle.flowFile, attributes);
@ -533,10 +543,10 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
private void handleParseFailure(final ConsumerRecord<byte[], byte[]> consumerRecord, final ProcessSession session, final Exception cause, final String message) {
// If we are unable to parse the data, we need to transfer it to 'parse failure' relationship
final Map<String, String> attributes = getAttributes(consumerRecord);
attributes.put(KafkaProcessorUtils.KAFKA_OFFSET, String.valueOf(consumerRecord.offset()));
attributes.put(KafkaProcessorUtils.KAFKA_TIMESTAMP, String.valueOf(consumerRecord.timestamp()));
attributes.put(KafkaProcessorUtils.KAFKA_PARTITION, String.valueOf(consumerRecord.partition()));
attributes.put(KafkaProcessorUtils.KAFKA_TOPIC, consumerRecord.topic());
attributes.put(KAFKA_OFFSET, String.valueOf(consumerRecord.offset()));
attributes.put(KAFKA_TIMESTAMP, String.valueOf(consumerRecord.timestamp()));
attributes.put(KAFKA_PARTITION, String.valueOf(consumerRecord.partition()));
attributes.put(KAFKA_TOPIC, consumerRecord.topic());
FlowFile failureFlowFile = session.create();
@ -546,7 +556,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);
@ -703,7 +713,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
throws IOException, SchemaNotFoundException, MalformedRecordException {
final byte[] key = consumerRecord.key() == null ? new byte[0] : consumerRecord.key();
if (KafkaProcessorUtils.KEY_AS_RECORD.getValue().equals(keyFormat)) {
if (KeyFormat.RECORD.getValue().equals(keyFormat)) {
if (key.length == 0) {
return new Tuple<>(EMPTY_SCHEMA_KEY_RECORD_FIELD, null);
}
@ -716,7 +726,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
final RecordField recordField = new RecordField("key", RecordFieldType.RECORD.getRecordDataType(record.getSchema()));
return new Tuple<>(recordField, record);
}
} else if (KafkaProcessorUtils.KEY_AS_STRING.getValue().equals(keyFormat)) {
} else if (KeyFormat.STRING.getValue().equals(keyFormat)) {
final RecordField recordField = new RecordField("key", RecordFieldType.STRING.getDataType());
final String keyString = ((key == null) ? null : new String(key, StandardCharsets.UTF_8));
return new Tuple<>(recordField, keyString);
@ -797,32 +807,32 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
private void populateAttributes(final BundleTracker tracker) {
final Map<String, String> kafkaAttrs = new HashMap<>();
kafkaAttrs.put(KafkaProcessorUtils.KAFKA_OFFSET, String.valueOf(tracker.initialOffset));
kafkaAttrs.put(KafkaProcessorUtils.KAFKA_TIMESTAMP, String.valueOf(tracker.initialTimestamp));
kafkaAttrs.put(KafkaProcessorUtils.KAFKA_MAX_OFFSET, String.valueOf(tracker.maxOffset));
kafkaAttrs.put(KAFKA_OFFSET, String.valueOf(tracker.initialOffset));
kafkaAttrs.put(KAFKA_TIMESTAMP, String.valueOf(tracker.initialTimestamp));
kafkaAttrs.put(KAFKA_MAX_OFFSET, String.valueOf(tracker.maxOffset));
if (tracker.leaderEpoch != null) {
kafkaAttrs.put(KafkaProcessorUtils.KAFKA_LEADER_EPOCH, String.valueOf(tracker.leaderEpoch));
kafkaAttrs.put(KAFKA_LEADER_EPOCH, String.valueOf(tracker.leaderEpoch));
}
kafkaAttrs.put(KafkaProcessorUtils.KAFKA_CONSUMER_GROUP_ID, kafkaConsumer.groupMetadata().groupId());
kafkaAttrs.put(KafkaProcessorUtils.KAFKA_CONSUMER_OFFSETS_COMMITTED, String.valueOf(commitOffsets));
kafkaAttrs.put(KAFKA_CONSUMER_GROUP_ID, kafkaConsumer.groupMetadata().groupId());
kafkaAttrs.put(KAFKA_CONSUMER_OFFSETS_COMMITTED, String.valueOf(commitOffsets));
// 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(KAFKA_PARTITION, String.valueOf(tracker.partition));
kafkaAttrs.put(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(KAFKA_COUNT, String.valueOf(tracker.totalRecords));
} else {
kafkaAttrs.put("record.count", String.valueOf(tracker.totalRecords));
}
@ -830,7 +840,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
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);
}

View File

@ -26,6 +26,7 @@ import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.kafka.shared.property.OutputStrategy;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;

View File

@ -43,11 +43,9 @@ import java.util.Set;
* Customized version of {@link org.apache.kafka.common.security.kerberos.KerberosLogin} which improves the re-login logic
* to avoid making system calls to kinit when the ticket cache is being used, and to avoid exiting the refresh thread so that
* it may recover if the ticket cache is externally refreshed.
*
* The re-login thread follows a similar approach used by NiFi's KerberosUser which attempts to call tgt.refresh()
* and falls back to a logout/login.
*
* The Kafka client is configured to use this login by setting SaslConfigs.SASL_LOGIN_CLASS in {@link KafkaProcessorUtils}
* The Kafka client is configured to use this login by setting SaslConfigs.SASL_LOGIN_CLASS
* when the SASL mechanism is GSSAPI.
*/
public class CustomKerberosLogin extends AbstractLogin {

View File

@ -1,646 +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.kerberos.KerberosUserService;
import org.apache.nifi.kerberos.SelfContainedKerberosUserService;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.security.krb.KerberosUser;
import org.apache.nifi.ssl.SSLContextService;
import org.apache.nifi.util.FormatUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.security.auth.login.AppConfigurationEntry;
import java.lang.reflect.Field;
import java.lang.reflect.Modifier;
import java.util.ArrayList;
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";
static final Logger LOGGER = LoggerFactory.getLogger(KafkaProcessorUtils.class);
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_MAX_OFFSET = "kafka.max.offset";
static final String KAFKA_LEADER_EPOCH = "kafka.leader.epoch";
static final String KAFKA_TIMESTAMP = "kafka.timestamp";
static final String KAFKA_COUNT = "kafka.count";
static final String KAFKA_CONSUMER_GROUP_ID = "kafka.consumer.id";
static final String KAFKA_CONSUMER_OFFSETS_COMMITTED = "kafka.consumer.offsets.committed";
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.");
static final AllowableValue PUBLISH_USE_VALUE = new AllowableValue(PublishStrategy.USE_VALUE.name(),
"Use Content as Record Value", "Write only the FlowFile content to the Kafka Record value.");
static final AllowableValue PUBLISH_USE_WRAPPER = new AllowableValue(PublishStrategy.USE_WRAPPER.name(),
"Use Wrapper", "Write the Kafka Record key, value, headers, and metadata into the Kafka Record value. (See processor usage for more information.)");
static final AllowableValue OUTPUT_USE_VALUE = new AllowableValue(OutputStrategy.USE_VALUE.name(),
"Use Content as Value", "Write only the Kafka Record value to the FlowFile record.");
static final AllowableValue OUTPUT_USE_WRAPPER = new AllowableValue(OutputStrategy.USE_WRAPPER.name(),
"Use Wrapper", "Write the Kafka Record key, value, headers, and metadata into the FlowFile record. (See processor usage for more information.)");
static final AllowableValue KEY_AS_STRING = new AllowableValue("string", "String", "Format the Kafka ConsumerRecord key as a UTF-8 string.");
static final AllowableValue KEY_AS_BYTE_ARRAY = new AllowableValue("byte-array", "Byte Array", "Format the Kafka ConsumerRecord key as a byte array.");
static final AllowableValue KEY_AS_RECORD = new AllowableValue("record", "Record", "Format the Kafka ConsumerRecord key as a record.");
public static final PropertyDescriptor BOOTSTRAP_SERVERS = new PropertyDescriptor.Builder()
.name(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)
.displayName("Kafka Brokers")
.description("A comma-separated list of known Kafka Brokers in the format <host>:<port>")
.required(true)
.addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.defaultValue("localhost:9092")
.build();
public static final PropertyDescriptor SECURITY_PROTOCOL = new PropertyDescriptor.Builder()
.name("security.protocol")
.displayName("Security Protocol")
.description("Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.")
.required(true)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.allowableValues(SEC_PLAINTEXT, SEC_SSL, SEC_SASL_PLAINTEXT, SEC_SASL_SSL)
.defaultValue(SEC_PLAINTEXT.getValue())
.build();
static final PropertyDescriptor SASL_MECHANISM = new PropertyDescriptor.Builder()
.name("sasl.mechanism")
.displayName("SASL Mechanism")
.description("The SASL mechanism to use for authentication. Corresponds to Kafka's 'sasl.mechanism' property.")
.required(true)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.allowableValues(SASL_MECHANISM_GSSAPI, SASL_MECHANISM_PLAIN, SASL_MECHANISM_SCRAM_SHA256, SASL_MECHANISM_SCRAM_SHA512)
.defaultValue(GSSAPI_VALUE)
.build();
public static final PropertyDescriptor JAAS_SERVICE_NAME = new PropertyDescriptor.Builder()
.name("sasl.kerberos.service.name")
.displayName("Kerberos Service Name")
.description("The service name that matches the primary name of the Kafka server configured in the broker JAAS file."
+ "This can be defined either in Kafka's JAAS config or in Kafka's config. "
+ "Corresponds to Kafka's 'security.protocol' property."
+ "It is ignored unless one of the SASL options of the <Security Protocol> are selected.")
.required(false)
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
static final PropertyDescriptor USER_PRINCIPAL = new PropertyDescriptor.Builder()
.name("sasl.kerberos.principal")
.displayName("Kerberos Principal")
.description("The Kerberos principal that will be used to connect to brokers. If not set, it is expected to set a JAAS configuration file "
+ "in the JVM properties defined in the bootstrap.conf file. This principal will be set into 'sasl.jaas.config' Kafka's property.")
.required(false)
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
static final PropertyDescriptor USER_KEYTAB = new PropertyDescriptor.Builder()
.name("sasl.kerberos.keytab")
.displayName("Kerberos Keytab")
.description("The Kerberos keytab that will be used to connect to brokers. If not set, it is expected to set a JAAS configuration file "
+ "in the JVM properties defined in the bootstrap.conf file. This principal will be set into 'sasl.jaas.config' Kafka's property.")
.required(false)
.identifiesExternalResource(ResourceCardinality.SINGLE, ResourceType.FILE)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
.name("sasl.username")
.displayName("Username")
.description("The username when the SASL Mechanism is " + PLAIN_VALUE + " or " + SCRAM_SHA256_VALUE + "/" + SCRAM_SHA512_VALUE)
.required(false)
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
.name("sasl.password")
.displayName("Password")
.description("The password for the given username when the SASL Mechanism is " + PLAIN_VALUE + " or " + SCRAM_SHA256_VALUE + "/" + SCRAM_SHA512_VALUE)
.required(false)
.sensitive(true)
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
static final PropertyDescriptor TOKEN_AUTH = new PropertyDescriptor.Builder()
.name("sasl.token.auth")
.displayName("Token Auth")
.description("When " + SASL_MECHANISM.getDisplayName() + " is " + SCRAM_SHA256_VALUE + " or " + SCRAM_SHA512_VALUE
+ ", this property indicates if token authentication should be used.")
.required(false)
.allowableValues("true", "false")
.defaultValue("false")
.build();
public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
.name("ssl.context.service")
.displayName("SSL Context Service")
.description("Specifies the SSL Context Service to use for communicating with Kafka.")
.required(false)
.identifiesControllerService(SSLContextService.class)
.build();
public static final PropertyDescriptor KERBEROS_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
.name("kerberos-credentials-service")
.displayName("Kerberos Credentials Service")
.description("Specifies the Kerberos Credentials Controller Service that should be used for authenticating with Kerberos")
.identifiesControllerService(KerberosCredentialsService.class)
.required(false)
.build();
public static final PropertyDescriptor SELF_CONTAINED_KERBEROS_USER_SERVICE = new PropertyDescriptor.Builder()
.name("kerberos-user-service")
.displayName("Kerberos User Service")
.description("Specifies the Kerberos User Controller Service that should be used for authenticating with Kerberos")
.identifiesControllerService(SelfContainedKerberosUserService.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();
public static final String JAVA_SECURITY_AUTH_LOGIN_CONFIG = "java.security.auth.login.config";
public static Collection<ValidationResult> validateCommonProperties(final ValidationContext validationContext) {
List<ValidationResult> results = new ArrayList<>();
final String securityProtocol = validationContext.getProperty(SECURITY_PROTOCOL).getValue();
final String saslMechanism = validationContext.getProperty(SASL_MECHANISM).getValue();
final KerberosUserService kerberosUserService = validationContext.getProperty(SELF_CONTAINED_KERBEROS_USER_SERVICE).asControllerService(KerberosUserService.class);
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());
}
if (kerberosUserService != null && (explicitPrincipal != null || explicitKeytab != null)) {
results.add(new ValidationResult.Builder()
.subject("Kerberos User")
.valid(false)
.explanation("Cannot specify both a Kerberos User Service and a principal/keytab")
.build());
}
if (kerberosUserService != null && credentialsService != null) {
results.add(new ValidationResult.Builder()
.subject("Kerberos User")
.valid(false)
.explanation("Cannot specify both a Kerberos User Service and a Kerberos Credentials Service")
.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());
}
final String jvmJaasConfigFile = System.getProperty(JAVA_SECURITY_AUTH_LOGIN_CONFIG);
if (kerberosUserService == null && resolvedPrincipal == null && resolvedKeytab == null && StringUtils.isBlank(jvmJaasConfigFile)) {
results.add(new ValidationResult.Builder()
.subject("Kerberos Credentials")
.valid(false)
.explanation("Kerberos credentials must be provided by a Kerberos Credentials Service, " +
"Kerberos User Service, explicit principal/keytab properties, or JVM JAAS configuration")
.build());
}
}
// validate that if SASL Mechanism is PLAIN or SCRAM, then username and password are both provided
if (SASL_MECHANISM_PLAIN.getValue().equals(saslMechanism)
|| SASL_MECHANISM_SCRAM_SHA256.getValue().equals(saslMechanism)
|| SASL_MECHANISM_SCRAM_SHA512.getValue().equals(saslMechanism)) {
final String username = validationContext.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
if (StringUtils.isBlank(username)) {
results.add(new ValidationResult.Builder()
.subject(USERNAME.getDisplayName())
.valid(false)
.explanation("A username is required when " + SASL_MECHANISM.getDisplayName()
+ " is " + PLAIN_VALUE + " or " + SCRAM_SHA256_VALUE + "/" + SCRAM_SHA512_VALUE)
.build());
}
final String password = validationContext.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
if (StringUtils.isBlank(password)) {
results.add(new ValidationResult.Builder()
.subject(PASSWORD.getDisplayName())
.valid(false)
.explanation("A password is required when " + SASL_MECHANISM.getDisplayName()
+ " is " + PLAIN_VALUE + " or " + SCRAM_SHA256_VALUE + "/" + SCRAM_SHA512_VALUE)
.build());
}
}
// If SSL or SASL_SSL then SSLContext Controller Service must be set.
final boolean sslProtocol = SEC_SSL.getValue().equals(securityProtocol) || SEC_SASL_SSL.getValue().equals(securityProtocol);
final boolean csSet = validationContext.getProperty(SSL_CONTEXT_SERVICE).isSet();
if (csSet && !sslProtocol) {
results.add(new ValidationResult.Builder()
.subject(SECURITY_PROTOCOL.getDisplayName())
.valid(false)
.explanation("If you set the SSL Controller Service you should also choose an SSL based security protocol.")
.build());
}
if (!csSet && sslProtocol) {
results.add(new ValidationResult.Builder()
.subject(SSL_CONTEXT_SERVICE.getDisplayName())
.valid(false)
.explanation("If you set to an SSL based protocol you need to set the SSL Controller Service")
.build());
}
final String enableAutoCommit = validationContext.getProperty(new PropertyDescriptor.Builder().name(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG).build()).getValue();
if (enableAutoCommit != null && !enableAutoCommit.toLowerCase().equals("false")) {
results.add(new ValidationResult.Builder().subject(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)
.explanation("Enable auto commit must be false. It is managed by the processor.").build());
}
final String keySerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG).build()).getValue();
if (keySerializer != null && !ByteArraySerializer.class.getName().equals(keySerializer)) {
results.add(new ValidationResult.Builder().subject(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)
.explanation("Key Serializer must be " + ByteArraySerializer.class.getName() + "' was '" + keySerializer + "'").build());
}
final String valueSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG).build()).getValue();
if (valueSerializer != null && !ByteArraySerializer.class.getName().equals(valueSerializer)) {
results.add(new ValidationResult.Builder().subject(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)
.explanation("Value Serializer must be " + ByteArraySerializer.class.getName() + "' was '" + valueSerializer + "'").build());
}
final String keyDeSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG).build()).getValue();
if (keyDeSerializer != null && !ByteArrayDeserializer.class.getName().equals(keyDeSerializer)) {
results.add(new ValidationResult.Builder().subject(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)
.explanation("Key De-Serializer must be '" + ByteArrayDeserializer.class.getName() + "' was '" + keyDeSerializer + "'").build());
}
final String valueDeSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG).build()).getValue();
if (valueDeSerializer != null && !ByteArrayDeserializer.class.getName().equals(valueDeSerializer)) {
results.add(new ValidationResult.Builder().subject(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG)
.explanation("Value De-Serializer must be " + ByteArrayDeserializer.class.getName() + "' was '" + valueDeSerializer + "'").build());
}
return results;
}
public static final class KafkaConfigValidator implements Validator {
final Class<?> classType;
public KafkaConfigValidator(final Class<?> classType) {
this.classType = classType;
}
@Override
public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
if (subject.startsWith(ConsumerPartitionsUtil.PARTITION_PROPERTY_NAME_PREFIX)) {
return new ValidationResult.Builder().valid(true).build();
}
final boolean knownValue = KafkaProcessorUtils.isStaticStringFieldNamePresent(subject, classType, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class);
return new ValidationResult.Builder().subject(subject).explanation("Must be a known configuration parameter for this kafka client").valid(knownValue).build();
}
}
/**
* Builds transit URI for provenance event. The transit URI will be in the
* form of &lt;security.protocol&gt;://&lt;bootstrap.servers&gt;/topic
*/
static String buildTransitURI(String securityProtocol, String brokers, String topic) {
StringBuilder builder = new StringBuilder();
builder.append(securityProtocol);
builder.append("://");
builder.append(brokers);
builder.append("/");
builder.append(topic);
return builder.toString();
}
static void buildCommonKafkaProperties(final ProcessContext context, final Class<?> kafkaConfigClass, final Map<String, Object> mapToPopulate) {
for (PropertyDescriptor propertyDescriptor : context.getProperties().keySet()) {
if (propertyDescriptor.equals(SSL_CONTEXT_SERVICE)) {
// Translate SSLContext Service configuration into Kafka properties
final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
if (sslContextService != null && sslContextService.isKeyStoreConfigured()) {
mapToPopulate.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, sslContextService.getKeyStoreFile());
mapToPopulate.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, sslContextService.getKeyStorePassword());
final String keyPass = sslContextService.getKeyPassword() == null ? sslContextService.getKeyStorePassword() : sslContextService.getKeyPassword();
mapToPopulate.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, keyPass);
mapToPopulate.put(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, sslContextService.getKeyStoreType());
}
if (sslContextService != null && sslContextService.isTrustStoreConfigured()) {
mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, sslContextService.getTrustStoreFile());
mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, sslContextService.getTrustStorePassword());
mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, sslContextService.getTrustStoreType());
}
}
String propertyName = propertyDescriptor.getName();
String propertyValue = propertyDescriptor.isExpressionLanguageSupported()
? context.getProperty(propertyDescriptor).evaluateAttributeExpressions().getValue()
: context.getProperty(propertyDescriptor).getValue();
if (propertyValue != null && !propertyName.equals(USER_PRINCIPAL.getName()) && !propertyName.equals(USER_KEYTAB.getName())
&& !propertyName.startsWith(ConsumerPartitionsUtil.PARTITION_PROPERTY_NAME_PREFIX)) {
// If the property name ends in ".ms" then it is a time period. We want to accept either an integer as number of milliseconds
// or the standard NiFi time period such as "5 secs"
if (propertyName.endsWith(".ms") && !StringUtils.isNumeric(propertyValue.trim())) { // kafka standard time notation
propertyValue = String.valueOf(FormatUtils.getTimeDuration(propertyValue.trim(), TimeUnit.MILLISECONDS));
}
if (isStaticStringFieldNamePresent(propertyName, kafkaConfigClass, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class)) {
mapToPopulate.put(propertyName, propertyValue);
}
}
}
final String securityProtocol = context.getProperty(SECURITY_PROTOCOL).getValue();
if (SEC_SASL_PLAINTEXT.getValue().equals(securityProtocol) || SEC_SASL_SSL.getValue().equals(securityProtocol)) {
setJaasConfig(mapToPopulate, context);
}
}
/**
* Method used to create a transactional id Supplier for KafkaProducer
*
* @param prefix String transactional id prefix, can be null
* @return A Supplier that generates transactional id
*/
static Supplier<String> getTransactionalIdSupplier(String prefix) {
return () -> (prefix == null ? "" : prefix) + UUID.randomUUID().toString();
}
/**
* Method used to configure the 'sasl.jaas.config' property based on KAFKA-4259<br />
* https://cwiki.apache.org/confluence/display/KAFKA/KIP-85%3A+Dynamic+JAAS+configuration+for+Kafka+clients<br />
* <br />
* It expects something with the following format: <br />
* <br />
* &lt;LoginModuleClass&gt; &lt;ControlFlag&gt; *(&lt;OptionName&gt;=&lt;OptionValue&gt;); <br />
* ControlFlag = required / requisite / sufficient / optional
*
* @param mapToPopulate Map of configuration properties
* @param context Context
*/
private static void setJaasConfig(Map<String, Object> mapToPopulate, ProcessContext context) {
final String saslMechanism = context.getProperty(SASL_MECHANISM).getValue();
switch (saslMechanism) {
case GSSAPI_VALUE:
setGssApiJaasConfig(mapToPopulate, context);
break;
case PLAIN_VALUE:
setPlainJaasConfig(mapToPopulate, context);
break;
case SCRAM_SHA256_VALUE:
case SCRAM_SHA512_VALUE:
setScramJaasConfig(mapToPopulate, context);
break;
default:
throw new IllegalStateException("Unknown " + SASL_MECHANISM.getDisplayName() + ": " + saslMechanism);
}
}
private static void setGssApiJaasConfig(final Map<String, Object> mapToPopulate, final ProcessContext context) {
final SelfContainedKerberosUserService kerberosUserService = context.getProperty(SELF_CONTAINED_KERBEROS_USER_SERVICE).asControllerService(SelfContainedKerberosUserService.class);
final String jaasConfig;
if (kerberosUserService == null) {
jaasConfig = createGssApiJaasConfig(context);
} else {
jaasConfig = createGssApiJaasConfig(kerberosUserService);
}
mapToPopulate.put(SaslConfigs.SASL_JAAS_CONFIG, jaasConfig);
mapToPopulate.put(SaslConfigs.SASL_LOGIN_CLASS, CustomKerberosLogin.class);
}
private static String createGssApiJaasConfig(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();
}
final String serviceName = context.getProperty(JAAS_SERVICE_NAME).evaluateAttributeExpressions().getValue();
return "com.sun.security.auth.module.Krb5LoginModule required "
+ "useTicketCache=false "
+ "renewTicket=true "
+ "serviceName=\"" + serviceName + "\" "
+ "useKeyTab=true "
+ "keyTab=\"" + keytab + "\" "
+ "principal=\"" + principal + "\";";
}
static String createGssApiJaasConfig(final SelfContainedKerberosUserService kerberosUserService) {
final KerberosUser kerberosUser = kerberosUserService.createKerberosUser();
final AppConfigurationEntry configEntry = kerberosUser.getConfigurationEntry();
final StringBuilder configBuilder = new StringBuilder(configEntry.getLoginModuleName())
.append(" ").append(getControlFlagValue(configEntry.getControlFlag()));
final Map<String, ?> options = configEntry.getOptions();
options.entrySet().forEach((entry) -> {
configBuilder.append(" ").append(entry.getKey()).append("=");
final Object value = entry.getValue();
if (value instanceof String) {
configBuilder.append("\"").append((String)value).append("\"");
} else {
configBuilder.append(value);
}
});
configBuilder.append(";");
return configBuilder.toString();
}
private static String getControlFlagValue(final AppConfigurationEntry.LoginModuleControlFlag controlFlag) {
if (controlFlag == AppConfigurationEntry.LoginModuleControlFlag.OPTIONAL) {
return "optional";
} else if (controlFlag == AppConfigurationEntry.LoginModuleControlFlag.REQUIRED) {
return "required";
} else if (controlFlag == AppConfigurationEntry.LoginModuleControlFlag.REQUISITE) {
return "requisite";
} else if (controlFlag == AppConfigurationEntry.LoginModuleControlFlag.SUFFICIENT) {
return "sufficient";
}
throw new IllegalStateException("Unknown control flag: " + controlFlag.toString());
}
private static void setPlainJaasConfig(final Map<String, Object> mapToPopulate, final ProcessContext context) {
final String username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
final String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
mapToPopulate.put(SaslConfigs.SASL_JAAS_CONFIG, "org.apache.kafka.common.security.plain.PlainLoginModule required "
+ "username=\"" + username + "\" "
+ "password=\"" + password + "\";");
}
private static void setScramJaasConfig(final Map<String, Object> mapToPopulate, final ProcessContext context) {
final String username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
final String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
final StringBuilder builder = new StringBuilder("org.apache.kafka.common.security.scram.ScramLoginModule required ")
.append("username=\"" + username + "\" ")
.append("password=\"" + password + "\"");
final Boolean tokenAuth = context.getProperty(TOKEN_AUTH).asBoolean();
if (tokenAuth != null && tokenAuth) {
builder.append(" tokenauth=\"true\"");
}
builder.append(";");
mapToPopulate.put(SaslConfigs.SASL_JAAS_CONFIG, builder.toString());
}
public static boolean isStaticStringFieldNamePresent(final String name, final Class<?>... classes) {
return KafkaProcessorUtils.getPublicStaticStringFieldValues(classes).contains(name);
}
private static Set<String> getPublicStaticStringFieldValues(final Class<?>... classes) {
final Set<String> strings = new HashSet<>();
for (final Class<?> classType : classes) {
for (final Field field : classType.getDeclaredFields()) {
if (Modifier.isPublic(field.getModifiers()) && Modifier.isStatic(field.getModifiers()) && field.getType().equals(String.class)) {
try {
strings.add(String.valueOf(field.get(null)));
} catch (IllegalArgumentException | IllegalAccessException ex) {
//ignore
}
}
}
}
return strings;
}
}

View File

@ -36,6 +36,15 @@ 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.property.PublishStrategy;
import org.apache.nifi.kafka.shared.transaction.TransactionIdSupplier;
import org.apache.nifi.kafka.shared.validation.KafkaClientCustomValidationFunction;
import org.apache.nifi.kafka.shared.property.FailureStrategy;
import org.apache.nifi.kafka.shared.property.provider.KafkaPropertyProvider;
import org.apache.nifi.kafka.shared.component.KafkaPublishComponent;
import org.apache.nifi.kafka.shared.property.provider.StandardKafkaPropertyProvider;
import org.apache.nifi.kafka.shared.validation.DynamicPropertyValidator;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.DataUnit;
@ -44,7 +53,6 @@ import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.VerifiableProcessor;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.record.path.RecordPath;
import org.apache.nifi.record.path.RecordPathResult;
@ -59,13 +67,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;
@ -81,10 +86,7 @@ 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;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.PUBLISH_USE_VALUE;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.PUBLISH_USE_WRAPPER;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_CONSUMER_OFFSETS_COMMITTED;
@Tags({"Apache", "Kafka", "Record", "csv", "json", "avro", "logs", "Put", "Send", "Message", "PubSub", "2.6"})
@CapabilityDescription("Sends the contents of a FlowFile as individual records to Apache Kafka using the Kafka 2.6 Producer API. "
@ -99,7 +101,7 @@ import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.PUBLIS
@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_6.class, ConsumeKafka_2_6.class, ConsumeKafkaRecord_2_6.class})
public class PublishKafkaRecord_2_6 extends AbstractProcessor implements VerifiableProcessor {
public class PublishKafkaRecord_2_6 extends AbstractProcessor implements KafkaPublishComponent, VerifiableProcessor {
protected static final String MSG_COUNT = "msg.count";
static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("all", "Guarantee Replicated Delivery",
@ -165,8 +167,8 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
.displayName("Publish Strategy")
.description("The format used to publish the incoming FlowFile record to Kafka.")
.required(true)
.defaultValue(PUBLISH_USE_VALUE.getValue())
.allowableValues(PUBLISH_USE_VALUE, PUBLISH_USE_WRAPPER)
.defaultValue(PublishStrategy.USE_VALUE.getValue())
.allowableValues(PublishStrategy.class)
.build();
static final PropertyDescriptor MESSAGE_KEY_FIELD = new Builder()
@ -175,7 +177,7 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
.description("The name of a field in the Input Records that should be used as the Key for the Kafka message.")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
.dependsOn(PUBLISH_STRATEGY, PUBLISH_USE_VALUE)
.dependsOn(PUBLISH_STRATEGY, PublishStrategy.USE_VALUE.getValue())
.required(false)
.build();
@ -256,7 +258,7 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
+ "If not specified, no FlowFile attributes will be added as headers.")
.addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
.expressionLanguageSupported(NONE)
.dependsOn(PUBLISH_STRATEGY, PUBLISH_USE_VALUE)
.dependsOn(PUBLISH_STRATEGY, PublishStrategy.USE_VALUE.getValue())
.required(false)
.build();
static final PropertyDescriptor USE_TRANSACTIONS = new Builder()
@ -294,7 +296,7 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
.displayName("Record Key Writer")
.description("The Record Key Writer to use for outgoing FlowFiles")
.identifiesControllerService(RecordSetWriterFactory.class)
.dependsOn(PUBLISH_STRATEGY, PUBLISH_USE_WRAPPER)
.dependsOn(PUBLISH_STRATEGY, PublishStrategy.USE_WRAPPER.getValue())
.build();
static final PropertyDescriptor RECORD_METADATA_STRATEGY = new Builder()
.name("Record Metadata Strategy")
@ -304,7 +306,7 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
.required(true)
.allowableValues(RECORD_METADATA_FROM_PROPERTIES, RECORD_METADATA_FROM_RECORD)
.defaultValue(RECORD_METADATA_FROM_PROPERTIES.getValue())
.dependsOn(PUBLISH_STRATEGY, PUBLISH_USE_WRAPPER)
.dependsOn(PUBLISH_STRATEGY, PublishStrategy.USE_WRAPPER.getValue())
.build();
static final Relationship REL_SUCCESS = new Relationship.Builder()
@ -325,30 +327,30 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
static {
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.add(KafkaProcessorUtils.BOOTSTRAP_SERVERS);
properties.add(BOOTSTRAP_SERVERS);
properties.add(TOPIC);
properties.add(RECORD_READER);
properties.add(RECORD_WRITER);
properties.add(USE_TRANSACTIONS);
properties.add(TRANSACTIONAL_ID_PREFIX);
properties.add(KafkaProcessorUtils.FAILURE_STRATEGY);
properties.add(FAILURE_STRATEGY);
properties.add(DELIVERY_GUARANTEE);
properties.add(PUBLISH_STRATEGY);
properties.add(RECORD_KEY_WRITER);
properties.add(RECORD_METADATA_STRATEGY);
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.SELF_CONTAINED_KERBEROS_USER_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(SELF_CONTAINED_KERBEROS_USER_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);
@ -380,7 +382,7 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
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();
@ -388,8 +390,8 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
@Override
protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
final List<ValidationResult> results = new ArrayList<>();
results.addAll(KafkaProcessorUtils.validateCommonProperties(validationContext));
final KafkaClientCustomValidationFunction validationFunction = new KafkaClientCustomValidationFunction();
final Collection<ValidationResult> results = validationFunction.apply(validationContext);
final boolean useTransactions = validationContext.getProperty(USE_TRANSACTIONS).asBoolean();
if (useTransactions) {
@ -444,21 +446,21 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
protected PublisherPool createPublisherPool(final ProcessContext context) {
final int maxMessageSize = context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue();
final long maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS).longValue();
final long maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
final String attributeNameRegex = context.getProperty(ATTRIBUTE_NAME_REGEX).getValue();
final Pattern attributeNamePattern = attributeNameRegex == null ? null : Pattern.compile(attributeNameRegex);
final boolean useTransactions = context.getProperty(USE_TRANSACTIONS).asBoolean();
final String transactionalIdPrefix = context.getProperty(TRANSACTIONAL_ID_PREFIX).evaluateAttributeExpressions().getValue();
Supplier<String> transactionalIdSupplier = KafkaProcessorUtils.getTransactionalIdSupplier(transactionalIdPrefix);
Supplier<String> transactionalIdSupplier = new TransactionIdSupplier(transactionalIdPrefix);
final PublishStrategy publishStrategy = PublishStrategy.valueOf(context.getProperty(PUBLISH_STRATEGY).getValue());
final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
final Charset charset = Charset.forName(charsetName);
final RecordSetWriterFactory recordKeyWriterFactory = context.getProperty(RECORD_KEY_WRITER).asControllerService(RecordSetWriterFactory.class);
final Map<String, Object> kafkaProperties = new HashMap<>();
KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
final KafkaPropertyProvider propertyProvider = new StandardKafkaPropertyProvider(ProducerConfig.class);
final Map<String, Object> kafkaProperties = propertyProvider.getProperties(context);
kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize));
@ -489,8 +491,8 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
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();
@ -535,9 +537,7 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
final Function<Record, Integer> partitioner = getPartitioner(context, flowFile);
try {
session.read(flowFile, new InputStreamCallback() {
@Override
public void process(final InputStream in) throws IOException {
session.read(flowFile, in -> {
try {
final RecordReader reader = readerFactory.createRecordReader(flowFile, in, getLogger());
final RecordSet recordSet = reader.createRecordSet();
@ -547,17 +547,15 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
} catch (final SchemaNotFoundException | MalformedRecordException e) {
throw new ProcessException(e);
}
}
});
// If consumer offsets haven't been committed, add them to the transaction.
if (useTransactions && "false".equals(flowFile.getAttribute(KafkaProcessorUtils.KAFKA_CONSUMER_OFFSETS_COMMITTED))) {
if (useTransactions && "false".equals(flowFile.getAttribute(KAFKA_CONSUMER_OFFSETS_COMMITTED))) {
PublishKafkaUtil.addConsumerOffsets(lease, flowFile, getLogger());
}
} catch (final Exception e) {
// The FlowFile will be obtained and the error logged below, when calling publishResult.getFailedFlowFiles()
lease.fail(flowFile, e);
continue;
}
}
@ -579,7 +577,7 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
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);
}
@ -623,7 +621,7 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
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);

View File

@ -27,12 +27,13 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.KAFKA_CONSUMER_GROUP_ID;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.KAFKA_LEADER_EPOCH;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.KAFKA_MAX_OFFSET;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.KAFKA_OFFSET;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.KAFKA_PARTITION;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.KAFKA_TOPIC;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_CONSUMER_GROUP_ID;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_CONSUMER_OFFSETS_COMMITTED;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_LEADER_EPOCH;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_MAX_OFFSET;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_OFFSET;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_PARTITION;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_TOPIC;
public class PublishKafkaUtil {
@ -50,7 +51,7 @@ public class PublishKafkaUtil {
// Check if any of the FlowFiles indicate that the consumer offsets have yet to be committed.
boolean offsetsCommitted = true;
for (final FlowFile flowFile : initialFlowFiles) {
if ("false".equals(flowFile.getAttribute(KafkaProcessorUtils.KAFKA_CONSUMER_OFFSETS_COMMITTED))) {
if ("false".equals(flowFile.getAttribute(KAFKA_CONSUMER_OFFSETS_COMMITTED))) {
offsetsCommitted = false;
break;
}
@ -89,14 +90,14 @@ public class PublishKafkaUtil {
* @param logger the processor's logger
*/
public static void addConsumerOffsets(final PublisherLease lease, final FlowFile flowFile, final ComponentLog logger) {
final String topic = flowFile.getAttribute(KafkaProcessorUtils.KAFKA_TOPIC);
final Long partition = getNumericAttribute(flowFile, KafkaProcessorUtils.KAFKA_PARTITION, logger);
Long maxOffset = getNumericAttribute(flowFile, KafkaProcessorUtils.KAFKA_MAX_OFFSET, logger);
final String topic = flowFile.getAttribute(KAFKA_TOPIC);
final Long partition = getNumericAttribute(flowFile, KAFKA_PARTITION, logger);
Long maxOffset = getNumericAttribute(flowFile, KAFKA_MAX_OFFSET, logger);
if (maxOffset == null) {
maxOffset = getNumericAttribute(flowFile, KafkaProcessorUtils.KAFKA_OFFSET, logger);
maxOffset = getNumericAttribute(flowFile, KAFKA_OFFSET, logger);
}
final Long epoch = getNumericAttribute(flowFile, KafkaProcessorUtils.KAFKA_LEADER_EPOCH, logger);
final Long epoch = getNumericAttribute(flowFile, KAFKA_LEADER_EPOCH, logger);
final String consumerGroupId = flowFile.getAttribute(KAFKA_CONSUMER_GROUP_ID);
if (topic == null || partition == null || maxOffset == null || consumerGroupId == null) {

View File

@ -35,6 +35,15 @@ 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.transaction.TransactionIdSupplier;
import org.apache.nifi.kafka.shared.validation.KafkaClientCustomValidationFunction;
import org.apache.nifi.kafka.shared.property.FailureStrategy;
import org.apache.nifi.kafka.shared.property.provider.KafkaPropertyProvider;
import org.apache.nifi.kafka.shared.component.KafkaPublishComponent;
import org.apache.nifi.kafka.shared.property.provider.StandardKafkaPropertyProvider;
import org.apache.nifi.kafka.shared.validation.DynamicPropertyValidator;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.DataUnit;
@ -43,19 +52,16 @@ import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.VerifiableProcessor;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.InputStreamCallback;
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 +72,7 @@ import java.util.function.Supplier;
import java.util.regex.Pattern;
import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.FAILURE_STRATEGY;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.FAILURE_STRATEGY_ROLLBACK;
import static org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute.KAFKA_CONSUMER_OFFSETS_COMMITTED;
@Tags({"Apache", "Kafka", "Put", "Send", "Message", "PubSub", "2.6"})
@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka using the Kafka 2.6 Producer API."
@ -83,7 +88,7 @@ import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.FAILUR
@WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Kafka for this FlowFile. This attribute is added only to "
+ "FlowFiles that are routed to success. If the <Message Demarcator> Property is not set, this will always be 1, but if the Property is set, it may "
+ "be greater than 1.")
public class PublishKafka_2_6 extends AbstractProcessor implements VerifiableProcessor {
public class PublishKafka_2_6 extends AbstractProcessor implements KafkaPublishComponent, VerifiableProcessor {
protected static final String MSG_COUNT = "msg.count";
static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("all", "Guarantee Replicated Delivery",
@ -178,7 +183,7 @@ public class PublishKafka_2_6 extends AbstractProcessor implements VerifiablePro
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)
@ -282,26 +287,26 @@ public class PublishKafka_2_6 extends AbstractProcessor implements VerifiablePro
static {
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.add(KafkaProcessorUtils.BOOTSTRAP_SERVERS);
properties.add(BOOTSTRAP_SERVERS);
properties.add(TOPIC);
properties.add(USE_TRANSACTIONS);
properties.add(TRANSACTIONAL_ID_PREFIX);
properties.add(MESSAGE_DEMARCATOR);
properties.add(KafkaProcessorUtils.FAILURE_STRATEGY);
properties.add(FAILURE_STRATEGY);
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.SELF_CONTAINED_KERBEROS_USER_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(SELF_CONTAINED_KERBEROS_USER_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(KEY);
properties.add(KEY_ATTRIBUTE_ENCODING);
properties.add(MAX_REQUEST_SIZE);
@ -334,7 +339,7 @@ public class PublishKafka_2_6 extends AbstractProcessor implements VerifiablePro
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();
@ -342,8 +347,8 @@ public class PublishKafka_2_6 extends AbstractProcessor implements VerifiablePro
@Override
protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
final List<ValidationResult> results = new ArrayList<>();
results.addAll(KafkaProcessorUtils.validateCommonProperties(validationContext));
final KafkaClientCustomValidationFunction validationFunction = new KafkaClientCustomValidationFunction();
final Collection<ValidationResult> results = validationFunction.apply(validationContext);
final boolean useTransactions = validationContext.getProperty(USE_TRANSACTIONS).asBoolean();
if (useTransactions) {
@ -384,19 +389,19 @@ public class PublishKafka_2_6 extends AbstractProcessor implements VerifiablePro
protected PublisherPool createPublisherPool(final ProcessContext context) {
final int maxMessageSize = context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue();
final long maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS).longValue();
final long maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
final String attributeNameRegex = context.getProperty(ATTRIBUTE_NAME_REGEX).getValue();
final Pattern attributeNamePattern = attributeNameRegex == null ? null : Pattern.compile(attributeNameRegex);
final boolean useTransactions = context.getProperty(USE_TRANSACTIONS).asBoolean();
final String transactionalIdPrefix = context.getProperty(TRANSACTIONAL_ID_PREFIX).evaluateAttributeExpressions().getValue();
Supplier<String> transactionalIdSupplier = KafkaProcessorUtils.getTransactionalIdSupplier(transactionalIdPrefix);
Supplier<String> transactionalIdSupplier = new TransactionIdSupplier(transactionalIdPrefix);
final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
final Charset charset = Charset.forName(charsetName);
final Map<String, Object> kafkaProperties = new HashMap<>();
KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
final KafkaPropertyProvider propertyProvider = new StandardKafkaPropertyProvider(ProducerConfig.class);
final Map<String, Object> kafkaProperties = propertyProvider.getProperties(context);
kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize));
@ -429,8 +434,8 @@ public class PublishKafka_2_6 extends AbstractProcessor implements VerifiablePro
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);
@ -465,17 +470,14 @@ public class PublishKafka_2_6 extends AbstractProcessor implements VerifiablePro
}
final Integer partition = getPartition(context, flowFile);
session.read(flowFile, new InputStreamCallback() {
@Override
public void process(final InputStream rawIn) throws IOException {
session.read(flowFile, rawIn -> {
try (final InputStream in = new BufferedInputStream(rawIn)) {
lease.publish(flowFile, in, messageKey, demarcatorBytes, topic, partition);
}
}
});
// If consumer offsets haven't been committed, add them to the transaction.
if (useTransactions && "false".equals(flowFile.getAttribute(KafkaProcessorUtils.KAFKA_CONSUMER_OFFSETS_COMMITTED))) {
if (useTransactions && "false".equals(flowFile.getAttribute(KAFKA_CONSUMER_OFFSETS_COMMITTED))) {
PublishKafkaUtil.addConsumerOffsets(lease, flowFile, getLogger());
}
}
@ -498,7 +500,7 @@ public class PublishKafka_2_6 extends AbstractProcessor implements VerifiablePro
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);
}
@ -513,7 +515,7 @@ public class PublishKafka_2_6 extends AbstractProcessor implements VerifiablePro
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);
@ -526,7 +528,7 @@ public class PublishKafka_2_6 extends AbstractProcessor implements VerifiablePro
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) {
@ -545,8 +547,7 @@ public class PublishKafka_2_6 extends AbstractProcessor implements VerifiablePro
final String partitionClass = context.getProperty(PARTITION_CLASS).getValue();
if (EXPRESSION_LANGUAGE_PARTITIONING.getValue().equals(partitionClass)) {
final String partition = context.getProperty(PARTITION).evaluateAttributeExpressions(flowFile).getValue();
final int hash = Objects.hashCode(partition);
return hash;
return Objects.hashCode(partition);
}
return null;

View File

@ -32,6 +32,7 @@ import org.apache.kafka.common.header.internals.RecordHeader;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.kafka.shared.property.PublishStrategy;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.schema.access.SchemaNotFoundException;
import org.apache.nifi.serialization.MalformedRecordException;

View File

@ -19,6 +19,7 @@ package org.apache.nifi.processors.kafka.pubsub;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.Producer;
import org.apache.nifi.kafka.shared.property.PublishStrategy;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.serialization.RecordSetWriterFactory;

View File

@ -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.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.KafkaClientCustomValidationFunction;
import org.apache.nifi.kafka.shared.validation.DynamicPropertyValidator;
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 +50,20 @@ 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.time.Duration;
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 2.6+ 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_2_6 extends AbstractControllerService implements RecordSinkService {
public class KafkaRecordSink_2_6 extends AbstractControllerService implements KafkaClientComponent, RecordSinkService {
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_6 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,17 @@ public class KafkaRecordSink_2_6 extends AbstractControllerService implements Re
private volatile Producer<byte[], byte[]> producer;
@Override
protected void init(final ControllerServiceInitializationContext context) throws InitializationException {
protected void init(final ControllerServiceInitializationContext context) {
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.add(KafkaProcessorUtils.BOOTSTRAP_SERVERS);
properties.add(BOOTSTRAP_SERVERS);
properties.add(TOPIC);
properties.add(RecordSinkService.RECORD_WRITER_FACTORY);
properties.add(DELIVERY_GUARANTEE);
properties.add(MESSAGE_HEADER_ENCODING);
properties.add(KafkaProcessorUtils.SECURITY_PROTOCOL);
properties.add(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE);
properties.add(KafkaProcessorUtils.JAAS_SERVICE_NAME);
properties.add(KafkaProcessorUtils.SSL_CONTEXT_SERVICE);
properties.add(SECURITY_PROTOCOL);
properties.add(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 +190,7 @@ public class KafkaRecordSink_2_6 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 +198,7 @@ public class KafkaRecordSink_2_6 extends AbstractControllerService implements Re
@Override
protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
return KafkaProcessorUtils.validateCommonProperties(validationContext);
return new KafkaClientCustomValidationFunction().apply(validationContext);
}
@OnEnabled
@ -217,11 +208,8 @@ public class KafkaRecordSink_2_6 extends AbstractControllerService implements Re
maxMessageSize = context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue();
maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue();
final Charset charset = Charset.forName(charsetName);
final Map<String, Object> kafkaProperties = new HashMap<>();
buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
final KafkaPropertyProvider propertyProvider = new StandardKafkaPropertyProvider(ProducerConfig.class);
final Map<String, Object> kafkaProperties = propertyProvider.getProperties(context);
kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize));
@ -295,94 +283,12 @@ public class KafkaRecordSink_2_6 extends AbstractControllerService implements Re
} catch (Exception e) {
throw new IOException("Failed to write metrics using record writer: " + e.getMessage(), e);
}
}
@OnDisabled
public void stop() throws IOException {
public void stop() {
if (producer != null) {
producer.close(maxAckWaitMillis, TimeUnit.MILLISECONDS);
}
}
static void buildCommonKafkaProperties(final ConfigurationContext context, final Class<?> kafkaConfigClass, final Map<String, Object> mapToPopulate) {
for (PropertyDescriptor propertyDescriptor : context.getProperties().keySet()) {
if (propertyDescriptor.equals(KafkaProcessorUtils.SSL_CONTEXT_SERVICE)) {
// Translate SSLContext Service configuration into Kafka properties
final SSLContextService sslContextService = context.getProperty(KafkaProcessorUtils.SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
if (sslContextService != null && sslContextService.isKeyStoreConfigured()) {
mapToPopulate.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, sslContextService.getKeyStoreFile());
mapToPopulate.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, sslContextService.getKeyStorePassword());
final String keyPass = sslContextService.getKeyPassword() == null ? sslContextService.getKeyStorePassword() : sslContextService.getKeyPassword();
mapToPopulate.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, keyPass);
mapToPopulate.put(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, sslContextService.getKeyStoreType());
}
if (sslContextService != null && sslContextService.isTrustStoreConfigured()) {
mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, sslContextService.getTrustStoreFile());
mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, sslContextService.getTrustStorePassword());
mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, sslContextService.getTrustStoreType());
}
}
String propertyName = propertyDescriptor.getName();
String propertyValue = propertyDescriptor.isExpressionLanguageSupported()
? context.getProperty(propertyDescriptor).evaluateAttributeExpressions().getValue()
: context.getProperty(propertyDescriptor).getValue();
if (propertyValue != null) {
// If the property name ends in ".ms" then it is a time period. We want to accept either an integer as number of milliseconds
// or the standard NiFi time period such as "5 secs"
if (propertyName.endsWith(".ms") && !StringUtils.isNumeric(propertyValue.trim())) { // kafka standard time notation
propertyValue = String.valueOf(FormatUtils.getTimeDuration(propertyValue.trim(), TimeUnit.MILLISECONDS));
}
if (KafkaProcessorUtils.isStaticStringFieldNamePresent(propertyName, kafkaConfigClass, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class)) {
mapToPopulate.put(propertyName, propertyValue);
}
}
}
String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue();
if (KafkaProcessorUtils.SEC_SASL_PLAINTEXT.getValue().equals(securityProtocol) || KafkaProcessorUtils.SEC_SASL_SSL.getValue().equals(securityProtocol)) {
setJaasConfig(mapToPopulate, context);
}
}
/**
* Method used to configure the 'sasl.jaas.config' property based on KAFKA-4259<br />
* https://cwiki.apache.org/confluence/display/KAFKA/KIP-85%3A+Dynamic+JAAS+configuration+for+Kafka+clients<br />
* <br />
* It expects something with the following format: <br />
* <br />
* &lt;LoginModuleClass&gt; &lt;ControlFlag&gt; *(&lt;OptionName&gt;=&lt;OptionValue&gt;); <br />
* ControlFlag = required / requisite / sufficient / optional
*
* @param mapToPopulate Map of configuration properties
* @param context Context
*/
private static void setJaasConfig(Map<String, Object> mapToPopulate, ConfigurationContext context) {
String keytab = null;
String principal = null;
// If the Kerberos Credentials Service is specified, we need to use its configuration, not the explicit properties for principal/keytab.
// The customValidate method ensures that only one can be set, so we know that the principal & keytab above are null.
final KerberosCredentialsService credentialsService = context.getProperty(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
if (credentialsService != null) {
principal = credentialsService.getPrincipal();
keytab = credentialsService.getKeytab();
}
String serviceName = context.getProperty(KafkaProcessorUtils.JAAS_SERVICE_NAME).evaluateAttributeExpressions().getValue();
if (StringUtils.isNotBlank(keytab) && StringUtils.isNotBlank(principal) && StringUtils.isNotBlank(serviceName)) {
mapToPopulate.put(SaslConfigs.SASL_JAAS_CONFIG, "com.sun.security.auth.module.Krb5LoginModule required "
+ "useTicketCache=false "
+ "renewTicket=true "
+ "serviceName=\"" + serviceName + "\" "
+ "useKeyTab=true "
+ "keyTab=\"" + keytab + "\" "
+ "principal=\"" + principal + "\";");
producer.close(Duration.ofMillis(maxAckWaitMillis));
}
}

View File

@ -56,7 +56,7 @@ public class ITConsumeKafka_2_6 {
}
};
final TestRunner runner = TestRunners.newTestRunner(proc);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
runner.setProperty(ConsumeKafka_2_6.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
runner.setProperty(ConsumeKafka_2_6.TOPICS, "foo,bar");
runner.setProperty(ConsumeKafka_2_6.GROUP_ID, groupName);
runner.setProperty(ConsumeKafka_2_6.AUTO_OFFSET_RESET, ConsumeKafka_2_6.OFFSET_EARLIEST);
@ -86,7 +86,7 @@ public class ITConsumeKafka_2_6 {
}
};
final TestRunner runner = TestRunners.newTestRunner(proc);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
runner.setProperty(ConsumeKafka_2_6.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
runner.setProperty(ConsumeKafka_2_6.TOPICS, "(fo.*)|(ba)");
runner.setProperty(ConsumeKafka_2_6.TOPIC_TYPE, "pattern");
runner.setProperty(ConsumeKafka_2_6.GROUP_ID, groupName);
@ -117,7 +117,7 @@ public class ITConsumeKafka_2_6 {
}
};
final TestRunner runner = TestRunners.newTestRunner(proc);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
runner.setProperty(ConsumeKafka_2_6.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
runner.setProperty(ConsumeKafka_2_6.TOPICS, "foo,bar");
runner.setProperty(ConsumeKafka_2_6.GROUP_ID, groupName);
runner.setProperty(ConsumeKafka_2_6.AUTO_OFFSET_RESET, ConsumeKafka_2_6.OFFSET_EARLIEST);

View File

@ -1,71 +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.nifi.kerberos.SelfContainedKerberosUserService;
import org.apache.nifi.security.krb.KerberosUser;
import org.junit.jupiter.api.Test;
import javax.security.auth.login.AppConfigurationEntry;
import java.util.HashMap;
import java.util.Map;
import java.util.function.Supplier;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class KafkaProcessorUtilsTest {
@Test
public void getTransactionalIdSupplierWithPrefix() {
Supplier<String> prefix = KafkaProcessorUtils.getTransactionalIdSupplier("prefix");
String id = prefix.get();
assertTrue(id.startsWith("prefix"));
assertEquals(42, id.length());
}
@Test
public void getTransactionalIdSupplierWithEmptyPrefix() {
Supplier<String> prefix = KafkaProcessorUtils.getTransactionalIdSupplier(null);
assertEquals(36, prefix.get().length() );
}
@Test
public void testCreateJaasConfigFromKerberosUser() {
final String loginModule = "com.sun.security.auth.module.Krb5LoginModule";
final AppConfigurationEntry.LoginModuleControlFlag controlFlag = AppConfigurationEntry.LoginModuleControlFlag.REQUIRED;
final Map<String,String> options = new HashMap<>();
options.put("option1", "value1");
options.put("option2", "value2");
final AppConfigurationEntry configEntry = new AppConfigurationEntry(loginModule, controlFlag, options);
final KerberosUser kerberosUser = mock(KerberosUser.class);
when(kerberosUser.getConfigurationEntry()).thenReturn(configEntry);
final SelfContainedKerberosUserService kerberosUserService = mock(SelfContainedKerberosUserService.class);
when(kerberosUserService.createKerberosUser()).thenReturn(kerberosUser);
final String jaasConfig = KafkaProcessorUtils.createGssApiJaasConfig(kerberosUserService);
assertNotNull(jaasConfig);
assertEquals("com.sun.security.auth.module.Krb5LoginModule required option1=\"value1\" option2=\"value2\";", jaasConfig);
}
}

View File

@ -35,6 +35,7 @@ import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.record.TimestampType;
import org.apache.nifi.json.JsonRecordSetWriter;
import org.apache.nifi.json.JsonTreeReader;
import org.apache.nifi.kafka.shared.property.OutputStrategy;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.reporting.InitializationException;
@ -57,6 +58,7 @@ import java.util.regex.Pattern;
import static java.nio.charset.StandardCharsets.ISO_8859_1;
import static java.nio.charset.StandardCharsets.UTF_8;
import static org.apache.nifi.kafka.shared.component.KafkaClientComponent.BOOTSTRAP_SERVERS;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
@ -75,7 +77,7 @@ public class TestConsumeKafkaMock {
/**
* Kafka server endpoint (mock) for test interactions.
*/
private static final String BOOTSTRAP_SERVER = "localhost:59092";
private static final String TEST_BOOTSTRAP_SERVER = "localhost:59092";
/**
* Ensure fresh data for each test run.
@ -418,7 +420,7 @@ public class TestConsumeKafkaMock {
};
final TestRunner runner = TestRunners.newTestRunner(processor);
runner.setValidateExpressionUsage(false);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, BOOTSTRAP_SERVER);
runner.setProperty(BOOTSTRAP_SERVERS, TEST_BOOTSTRAP_SERVER);
runner.setProperty("topic", topic);
runner.setProperty("topic_type", "names");
runner.setProperty(ConsumerConfig.GROUP_ID_CONFIG, group);

View File

@ -16,6 +16,8 @@
*/
package org.apache.nifi.processors.kafka.pubsub;
import org.apache.nifi.kafka.shared.property.KeyFormat;
import org.apache.nifi.kafka.shared.property.OutputStrategy;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processors.kafka.pubsub.util.MockRecordParser;
@ -28,20 +30,16 @@ import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.KEY_AS_RECORD;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.OUTPUT_USE_VALUE;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.OUTPUT_USE_WRAPPER;
import static org.apache.nifi.kafka.shared.component.KafkaClientComponent.BOOTSTRAP_SERVERS;
import static org.mockito.Mockito.mock;
public class TestConsumeKafkaRecordKey_2_6 {
private ConsumerLease mockLease = null;
private ConsumerPool mockConsumerPool = null;
private TestRunner runner;
@BeforeEach
public void setup() throws InitializationException {
mockLease = mock(ConsumerLease.class);
mockConsumerPool = mock(ConsumerPool.class);
ConsumeKafkaRecord_2_6 proc = new ConsumeKafkaRecord_2_6() {
@ -52,7 +50,7 @@ public class TestConsumeKafkaRecordKey_2_6 {
};
runner = TestRunners.newTestRunner(proc);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
runner.setProperty(BOOTSTRAP_SERVERS, "okeydokey:1234");
final String readerId = "record-reader";
final MockRecordParser readerService = new MockRecordParser();
@ -78,13 +76,13 @@ public class TestConsumeKafkaRecordKey_2_6 {
runner.assertValid();
runner.setProperty(ConsumeKafkaRecord_2_6.OUTPUT_STRATEGY, "foo");
runner.assertNotValid();
runner.setProperty(ConsumeKafkaRecord_2_6.OUTPUT_STRATEGY, OUTPUT_USE_VALUE);
runner.setProperty(ConsumeKafkaRecord_2_6.OUTPUT_STRATEGY, OutputStrategy.USE_VALUE.getValue());
runner.assertValid();
runner.setProperty(ConsumeKafkaRecord_2_6.OUTPUT_STRATEGY, OUTPUT_USE_WRAPPER);
runner.setProperty(ConsumeKafkaRecord_2_6.OUTPUT_STRATEGY, OutputStrategy.USE_WRAPPER.getValue());
runner.assertValid();
runner.setProperty(ConsumeKafkaRecord_2_6.KEY_FORMAT, "foo");
runner.assertNotValid();
runner.setProperty(ConsumeKafkaRecord_2_6.KEY_FORMAT, KEY_AS_RECORD);
runner.setProperty(ConsumeKafkaRecord_2_6.KEY_FORMAT, KeyFormat.RECORD.getValue());
runner.assertValid();
runner.setProperty(ConsumeKafkaRecord_2_6.KEY_RECORD_READER, "no-record-reader");
runner.assertNotValid();

View File

@ -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_6 {
};
runner = TestRunners.newTestRunner(proc);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
runner.setProperty(ConsumeKafkaRecord_2_6.BOOTSTRAP_SERVERS, "okeydokey:1234");
final String readerId = "record-reader";
final MockRecordParser readerService = new MockRecordParser();
@ -77,24 +79,20 @@ public class TestConsumeKafkaRecord_2_6 {
}
@Test
public void validateCustomValidatorSettings() throws Exception {
public void validateCustomValidatorSettings() {
runner.setProperty(ConsumeKafkaRecord_2_6.TOPICS, "foo");
runner.setProperty(ConsumeKafkaRecord_2_6.GROUP_ID, "foo");
runner.setProperty(ConsumeKafkaRecord_2_6.AUTO_OFFSET_RESET, ConsumeKafkaRecord_2_6.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_6.TOPICS, "foo");
runner.setProperty(ConsumeKafkaRecord_2_6.GROUP_ID, "foo");
runner.setProperty(ConsumeKafkaRecord_2_6.AUTO_OFFSET_RESET, ConsumeKafkaRecord_2_6.OFFSET_EARLIEST);
@ -116,7 +114,7 @@ public class TestConsumeKafkaRecord_2_6 {
}
@Test
public void validateGetAllMessages() throws Exception {
public void validateGetAllMessages() {
String groupName = "validateGetAllMessages";
when(mockConsumerPool.obtainConsumer(any(), any())).thenReturn(mockLease);
@ -138,7 +136,7 @@ public class TestConsumeKafkaRecord_2_6 {
}
@Test
public void validateGetAllMessagesPattern() throws Exception {
public void validateGetAllMessagesPattern() {
String groupName = "validateGetAllMessagesPattern";
when(mockConsumerPool.obtainConsumer(any(), any())).thenReturn(mockLease);
@ -188,19 +186,19 @@ public class TestConsumeKafkaRecord_2_6 {
runner.setProperty(ConsumeKafkaRecord_2_6.GROUP_ID, "foo");
runner.setProperty(ConsumeKafkaRecord_2_6.AUTO_OFFSET_RESET, ConsumeKafkaRecord_2_6.OFFSET_EARLIEST);
runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT);
runner.setProperty(ConsumeKafkaRecord_2_6.SECURITY_PROTOCOL, SecurityProtocol.SASL_PLAINTEXT.name());
runner.assertNotValid();
runner.setProperty(KafkaProcessorUtils.JAAS_SERVICE_NAME, "kafka");
runner.setProperty(ConsumeKafkaRecord_2_6.KERBEROS_SERVICE_NAME, "kafka");
runner.assertNotValid();
runner.setProperty(KafkaProcessorUtils.USER_PRINCIPAL, "nifi@APACHE.COM");
runner.setProperty(ConsumeKafkaRecord_2_6.KERBEROS_PRINCIPAL, "nifi@APACHE.COM");
runner.assertNotValid();
runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "not.A.File");
runner.setProperty(ConsumeKafkaRecord_2_6.KERBEROS_KEYTAB, "not.A.File");
runner.assertNotValid();
runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "src/test/resources/server.properties");
runner.setProperty(ConsumeKafkaRecord_2_6.KERBEROS_KEYTAB, "src/test/resources/server.properties");
runner.assertValid();
}
@ -210,19 +208,19 @@ public class TestConsumeKafkaRecord_2_6 {
runner.setProperty(ConsumeKafkaRecord_2_6.GROUP_ID, "foo");
runner.setProperty(ConsumeKafkaRecord_2_6.AUTO_OFFSET_RESET, ConsumeKafkaRecord_2_6.OFFSET_EARLIEST);
runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT);
runner.setProperty(ConsumeKafkaRecord_2_6.SECURITY_PROTOCOL, SecurityProtocol.SASL_PLAINTEXT.name());
runner.assertNotValid();
runner.setProperty(KafkaProcessorUtils.SASL_MECHANISM, KafkaProcessorUtils.PLAIN_VALUE);
runner.setProperty(ConsumeKafkaRecord_2_6.SASL_MECHANISM, SaslMechanism.PLAIN.getValue());
runner.assertNotValid();
runner.setProperty(KafkaProcessorUtils.USERNAME, "user1");
runner.setProperty(ConsumeKafkaRecord_2_6.SASL_USERNAME, "user1");
runner.assertNotValid();
runner.setProperty(KafkaProcessorUtils.PASSWORD, "password");
runner.setProperty(ConsumeKafkaRecord_2_6.SASL_PASSWORD, "password");
runner.assertValid();
runner.removeProperty(KafkaProcessorUtils.USERNAME);
runner.removeProperty(ConsumeKafkaRecord_2_6.SASL_USERNAME);
runner.assertNotValid();
}
@ -232,19 +230,19 @@ public class TestConsumeKafkaRecord_2_6 {
runner.setProperty(ConsumeKafkaRecord_2_6.GROUP_ID, "foo");
runner.setProperty(ConsumeKafkaRecord_2_6.AUTO_OFFSET_RESET, ConsumeKafkaRecord_2_6.OFFSET_EARLIEST);
runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT);
runner.setProperty(ConsumeKafkaRecord_2_6.SECURITY_PROTOCOL, SecurityProtocol.SASL_PLAINTEXT.name());
runner.assertNotValid();
runner.setProperty(KafkaProcessorUtils.SASL_MECHANISM, KafkaProcessorUtils.SCRAM_SHA256_VALUE);
runner.setProperty(ConsumeKafkaRecord_2_6.SASL_MECHANISM, SaslMechanism.SCRAM_SHA_256.getValue());
runner.assertNotValid();
runner.setProperty(KafkaProcessorUtils.USERNAME, "user1");
runner.setProperty(ConsumeKafkaRecord_2_6.SASL_USERNAME, "user1");
runner.assertNotValid();
runner.setProperty(KafkaProcessorUtils.PASSWORD, "password");
runner.setProperty(ConsumeKafkaRecord_2_6.SASL_PASSWORD, "password");
runner.assertValid();
runner.removeProperty(KafkaProcessorUtils.USERNAME);
runner.removeProperty(ConsumeKafkaRecord_2_6.SASL_USERNAME);
runner.assertNotValid();
}
@ -254,19 +252,19 @@ public class TestConsumeKafkaRecord_2_6 {
runner.setProperty(ConsumeKafkaRecord_2_6.GROUP_ID, "foo");
runner.setProperty(ConsumeKafkaRecord_2_6.AUTO_OFFSET_RESET, ConsumeKafkaRecord_2_6.OFFSET_EARLIEST);
runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT);
runner.setProperty(ConsumeKafkaRecord_2_6.SECURITY_PROTOCOL, SecurityProtocol.SASL_PLAINTEXT.name());
runner.assertNotValid();
runner.setProperty(KafkaProcessorUtils.SASL_MECHANISM, KafkaProcessorUtils.SCRAM_SHA512_VALUE);
runner.setProperty(ConsumeKafkaRecord_2_6.SASL_MECHANISM, SaslMechanism.SCRAM_SHA_512.getValue());
runner.assertNotValid();
runner.setProperty(KafkaProcessorUtils.USERNAME, "user1");
runner.setProperty(ConsumeKafkaRecord_2_6.SASL_USERNAME, "user1");
runner.assertNotValid();
runner.setProperty(KafkaProcessorUtils.PASSWORD, "password");
runner.setProperty(ConsumeKafkaRecord_2_6.SASL_PASSWORD, "password");
runner.assertValid();
runner.removeProperty(KafkaProcessorUtils.USERNAME);
runner.removeProperty(ConsumeKafkaRecord_2_6.SASL_USERNAME);
runner.assertNotValid();
}
@ -276,7 +274,7 @@ public class TestConsumeKafkaRecord_2_6 {
runner.setProperty(ConsumeKafkaRecord_2_6.GROUP_ID, "foo");
runner.setProperty(ConsumeKafkaRecord_2_6.AUTO_OFFSET_RESET, ConsumeKafkaRecord_2_6.OFFSET_EARLIEST);
runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_PLAINTEXT);
runner.setProperty(ConsumeKafkaRecord_2_6.SECURITY_PROTOCOL, SecurityProtocol.PLAINTEXT.name());
runner.assertValid();
}

View File

@ -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.kerberos.KerberosCredentialsService;
import org.apache.nifi.kerberos.KerberosUserService;
import org.apache.nifi.kerberos.SelfContainedKerberosUserService;
@ -44,47 +46,41 @@ public class TestConsumeKafka_2_6 {
}
@Test
public void validateCustomValidatorSettings() throws Exception {
public void validateCustomValidatorSettings() {
ConsumeKafka_2_6 consumeKafka = new ConsumeKafka_2_6();
TestRunner runner = TestRunners.newTestRunner(consumeKafka);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
runner.setProperty(ConsumeKafka_2_6.BOOTSTRAP_SERVERS, "okeydokey:1234");
runner.setProperty(ConsumeKafka_2_6.TOPICS, "foo");
runner.setProperty(ConsumeKafka_2_6.GROUP_ID, "foo");
runner.setProperty(ConsumeKafka_2_6.AUTO_OFFSET_RESET, ConsumeKafka_2_6.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_6 consumeKafka = new ConsumeKafka_2_6();
TestRunner runner = TestRunners.newTestRunner(consumeKafka);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
runner.setProperty(ConsumeKafka_2_6.BOOTSTRAP_SERVERS, "okeydokey:1234");
runner.setProperty(ConsumeKafka_2_6.TOPICS, "foo");
runner.setProperty(ConsumeKafka_2_6.GROUP_ID, "foo");
runner.setProperty(ConsumeKafka_2_6.AUTO_OFFSET_RESET, ConsumeKafka_2_6.OFFSET_EARLIEST);
runner.removeProperty(ConsumeKafka_2_6.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_6.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_6.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"));
}
@ -92,48 +88,48 @@ public class TestConsumeKafka_2_6 {
public void testJaasGssApiConfiguration() throws Exception {
ConsumeKafka_2_6 consumeKafka = new ConsumeKafka_2_6();
TestRunner runner = TestRunners.newTestRunner(consumeKafka);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
runner.setProperty(ConsumeKafka_2_6.BOOTSTRAP_SERVERS, "okeydokey:1234");
runner.setProperty(ConsumeKafka_2_6.TOPICS, "foo");
runner.setProperty(ConsumeKafka_2_6.GROUP_ID, "foo");
runner.setProperty(ConsumeKafka_2_6.AUTO_OFFSET_RESET, ConsumeKafka_2_6.OFFSET_EARLIEST);
runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT);
runner.setProperty(KafkaProcessorUtils.SASL_MECHANISM, KafkaProcessorUtils.GSSAPI_VALUE);
runner.setProperty(ConsumeKafka_2_6.SECURITY_PROTOCOL, SecurityProtocol.SASL_PLAINTEXT.name());
runner.setProperty(ConsumeKafka_2_6.SASL_MECHANISM, SaslMechanism.GSSAPI.getValue());
runner.assertNotValid();
runner.setProperty(KafkaProcessorUtils.JAAS_SERVICE_NAME, "kafka");
runner.setProperty(ConsumeKafka_2_6.KERBEROS_SERVICE_NAME, "kafka");
runner.assertNotValid();
runner.setProperty(KafkaProcessorUtils.USER_PRINCIPAL, "nifi@APACHE.COM");
runner.setProperty(ConsumeKafka_2_6.KERBEROS_PRINCIPAL, "nifi@APACHE.COM");
runner.assertNotValid();
runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "not.A.File");
runner.setProperty(ConsumeKafka_2_6.KERBEROS_KEYTAB, "not.A.File");
runner.assertNotValid();
runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "src/test/resources/server.properties");
runner.setProperty(ConsumeKafka_2_6.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_6.KERBEROS_PRINCIPAL, "${principal}");
runner.setProperty(ConsumeKafka_2_6.KERBEROS_KEYTAB, "${keytab}");
runner.setProperty(ConsumeKafka_2_6.KERBEROS_SERVICE_NAME, "${service}");
runner.assertValid();
final KerberosUserService kerberosUserService = enableKerberosUserService(runner);
runner.setProperty(KafkaProcessorUtils.SELF_CONTAINED_KERBEROS_USER_SERVICE, kerberosUserService.getIdentifier());
runner.setProperty(ConsumeKafka_2_6.SELF_CONTAINED_KERBEROS_USER_SERVICE, kerberosUserService.getIdentifier());
runner.assertNotValid();
runner.removeProperty(KafkaProcessorUtils.USER_PRINCIPAL);
runner.removeProperty(KafkaProcessorUtils.USER_KEYTAB);
runner.removeProperty(ConsumeKafka_2_6.KERBEROS_PRINCIPAL);
runner.removeProperty(ConsumeKafka_2_6.KERBEROS_KEYTAB);
runner.assertValid();
final KerberosCredentialsService kerberosCredentialsService = enabledKerberosCredentialsService(runner);
runner.setProperty(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE, kerberosCredentialsService.getIdentifier());
runner.setProperty(ConsumeKafka_2_6.KERBEROS_CREDENTIALS_SERVICE, kerberosCredentialsService.getIdentifier());
runner.assertNotValid();
runner.removeProperty(KafkaProcessorUtils.SELF_CONTAINED_KERBEROS_USER_SERVICE);
runner.removeProperty(ConsumeKafka_2_6.SELF_CONTAINED_KERBEROS_USER_SERVICE);
runner.assertValid();
}

View File

@ -27,6 +27,8 @@ import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.header.Headers;
import org.apache.nifi.json.JsonRecordSetWriter;
import org.apache.nifi.json.JsonTreeReader;
import org.apache.nifi.kafka.shared.property.PublishStrategy;
import org.apache.nifi.kafka.shared.transaction.TransactionIdSupplier;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.ProcessContext;
@ -315,7 +317,7 @@ public class TestPublishKafkaMock {
final Pattern attributeNamePattern = attributeNameRegex == null ? null : Pattern.compile(attributeNameRegex);
final boolean useTransactions = context.getProperty("use-transactions").asBoolean();
final String transactionalIdPrefix = context.getProperty("transactional-id-prefix").evaluateAttributeExpressions().getValue();
Supplier<String> transactionalIdSupplier = KafkaProcessorUtils.getTransactionalIdSupplier(transactionalIdPrefix);
Supplier<String> transactionalIdSupplier = new TransactionIdSupplier(transactionalIdPrefix);
final PublishStrategy publishStrategy = PublishStrategy.valueOf(context.getProperty("publish-strategy").getValue());
final String charsetName = context.getProperty("message-header-encoding").evaluateAttributeExpressions().getValue();
final Charset charset = Charset.forName(charsetName);

View File

@ -34,6 +34,8 @@ import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.header.internals.RecordHeader;
import org.apache.nifi.json.JsonRecordSetWriter;
import org.apache.nifi.json.JsonTreeReader;
import org.apache.nifi.kafka.shared.property.PublishStrategy;
import org.apache.nifi.kafka.shared.transaction.TransactionIdSupplier;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.ProcessContext;
@ -226,7 +228,7 @@ public class TestPublishKafkaMockParameterized {
final Pattern attributeNamePattern = attributeNameRegex == null ? null : Pattern.compile(attributeNameRegex);
final boolean useTransactions = context.getProperty("use-transactions").asBoolean();
final String transactionalIdPrefix = context.getProperty("transactional-id-prefix").evaluateAttributeExpressions().getValue();
Supplier<String> transactionalIdSupplier = KafkaProcessorUtils.getTransactionalIdSupplier(transactionalIdPrefix);
Supplier<String> transactionalIdSupplier = new TransactionIdSupplier(transactionalIdPrefix);
final PublishStrategy publishStrategy = PublishStrategy.valueOf(context.getProperty("publish-strategy").getValue());
final String charsetName = context.getProperty("message-header-encoding").evaluateAttributeExpressions().getValue();
final Charset charset = Charset.forName(charsetName);

View File

@ -18,6 +18,7 @@
package org.apache.nifi.processors.kafka.pubsub;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.kafka.shared.property.PublishStrategy;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processors.kafka.pubsub.util.MockRecordParser;
import org.apache.nifi.reporting.InitializationException;
@ -35,8 +36,6 @@ import org.mockito.Mockito;
import java.io.IOException;
import java.util.function.Function;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.PUBLISH_USE_VALUE;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.PUBLISH_USE_WRAPPER;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.nullable;
import static org.mockito.Mockito.mock;
@ -90,9 +89,9 @@ public class TestPublishKafkaRecordKey_2_6 {
runner.assertValid();
runner.setProperty(PublishKafkaRecord_2_6.PUBLISH_STRATEGY, "foo");
runner.assertNotValid();
runner.setProperty(PublishKafkaRecord_2_6.PUBLISH_STRATEGY, PUBLISH_USE_VALUE);
runner.setProperty(PublishKafkaRecord_2_6.PUBLISH_STRATEGY, PublishStrategy.USE_VALUE.getValue());
runner.assertValid();
runner.setProperty(PublishKafkaRecord_2_6.PUBLISH_STRATEGY, PUBLISH_USE_WRAPPER);
runner.setProperty(PublishKafkaRecord_2_6.PUBLISH_STRATEGY, PublishStrategy.USE_WRAPPER.getValue());
runner.assertValid();
runner.setProperty(PublishKafkaRecord_2_6.RECORD_KEY_WRITER, "no-record-writer");
runner.assertNotValid();

View File

@ -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_6 {
@Test
public void testSingleFailureWithRollback() throws IOException {
runner.setProperty(KafkaProcessorUtils.FAILURE_STRATEGY, KafkaProcessorUtils.FAILURE_STRATEGY_ROLLBACK);
runner.setProperty(PublishKafkaRecord_2_6.FAILURE_STRATEGY, FailureStrategy.ROLLBACK.getValue());
final MockFlowFile flowFile = runner.enqueue("John Doe, 48");
@ -189,7 +190,7 @@ public class TestPublishKafkaRecord_2_6 {
@Test
public void testFailureWhenCreatingTransactionWithRollback() {
runner.setProperty(KafkaProcessorUtils.FAILURE_STRATEGY, KafkaProcessorUtils.FAILURE_STRATEGY_ROLLBACK);
runner.setProperty(PublishKafkaRecord_2_6.FAILURE_STRATEGY, FailureStrategy.ROLLBACK.getValue());
runner.enqueue("John Doe, 48");
doAnswer((Answer<Object>) invocationOnMock -> {
@ -224,7 +225,7 @@ public class TestPublishKafkaRecord_2_6 {
@Test
public void testMultipleFailuresWithRollback() throws IOException {
runner.setProperty(KafkaProcessorUtils.FAILURE_STRATEGY, KafkaProcessorUtils.FAILURE_STRATEGY_ROLLBACK);
runner.setProperty(PublishKafkaRecord_2_6.FAILURE_STRATEGY, FailureStrategy.ROLLBACK.getValue());
final Set<FlowFile> flowFiles = new HashSet<>();
flowFiles.add(runner.enqueue("John Doe, 48"));
flowFiles.add(runner.enqueue("John Doe, 48"));

View File

@ -19,6 +19,7 @@ package org.apache.nifi.processors.kafka.pubsub;
import org.apache.kafka.common.errors.ProducerFencedException;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.kafka.shared.property.FailureStrategy;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
@ -139,7 +140,7 @@ public class TestPublishKafka_2_6 {
@Test
public void testSingleFailureWithRollback() throws IOException {
runner.setProperty(KafkaProcessorUtils.FAILURE_STRATEGY, KafkaProcessorUtils.FAILURE_STRATEGY_ROLLBACK);
runner.setProperty(PublishKafka_2_6.FAILURE_STRATEGY, FailureStrategy.ROLLBACK.getValue());
final MockFlowFile flowFile = runner.enqueue("hello world");
when(mockLease.complete()).thenReturn(createFailurePublishResult(flowFile));
@ -155,7 +156,7 @@ public class TestPublishKafka_2_6 {
@Test
public void testMultipleFailuresWithRollback() throws IOException {
runner.setProperty(KafkaProcessorUtils.FAILURE_STRATEGY, KafkaProcessorUtils.FAILURE_STRATEGY_ROLLBACK);
runner.setProperty(PublishKafka_2_6.FAILURE_STRATEGY, FailureStrategy.ROLLBACK.getValue());
final Set<FlowFile> flowFiles = new HashSet<>();
flowFiles.add(runner.enqueue("hello world"));

View File

@ -27,10 +27,11 @@ 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.component.KafkaClientComponent;
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 +139,13 @@ public class TestKafkaRecordSink_2_6 {
when(context.getProperty(KafkaRecordSink_2_6.MESSAGE_HEADER_ENCODING)).thenReturn(charEncodingValue);
final PropertyValue securityValue = Mockito.mock(StandardPropertyValue.class);
when(securityValue.getValue()).thenReturn(KafkaProcessorUtils.SEC_SASL_PLAINTEXT.getValue());
when(context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL)).thenReturn(securityValue);
when(securityValue.getValue()).thenReturn(SecurityProtocol.PLAINTEXT.name());
when(context.getProperty(KafkaRecordSink_2_6.SECURITY_PROTOCOL)).thenReturn(securityValue);
final PropertyValue jaasValue = Mockito.mock(StandardPropertyValue.class);
when(jaasValue.evaluateAttributeExpressions()).thenReturn(jaasValue);
when(jaasValue.getValue()).thenReturn(null);
when(context.getProperty(KafkaProcessorUtils.JAAS_SERVICE_NAME)).thenReturn(jaasValue);
when(context.getProperty(KafkaRecordSink_2_6.KERBEROS_SERVICE_NAME)).thenReturn(jaasValue);
Map<PropertyDescriptor, String> propertyMap = new HashMap<>();
propertyMap.put(KafkaRecordSink_2_6.TOPIC, KafkaRecordSink_2_6.TOPIC.getName());
@ -160,9 +161,9 @@ public class TestKafkaRecordSink_2_6 {
MockRecordWriter writer = new MockRecordWriter(null, false);
when(context.getProperty(RecordSinkService.RECORD_WRITER_FACTORY)).thenReturn(pValue);
when(pValue.asControllerService(RecordSetWriterFactory.class)).thenReturn(writer);
when(context.getProperty(KafkaProcessorUtils.SSL_CONTEXT_SERVICE)).thenReturn(pValue);
when(context.getProperty(KafkaRecordSink_2_6.SSL_CONTEXT_SERVICE)).thenReturn(pValue);
when(pValue.asControllerService(SSLContextService.class)).thenReturn(null);
when(context.getProperty(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE)).thenReturn(pValue);
when(context.getProperty(KafkaClientComponent.KERBEROS_CREDENTIALS_SERVICE)).thenReturn(pValue);
when(pValue.asControllerService(KerberosCredentialsService.class)).thenReturn(null);
final ControllerServiceInitializationContext initContext = new MockControllerServiceInitializationContext(task, UUID.randomUUID().toString(), logger, stateManager);

View File

@ -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

View File

@ -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

View File

@ -0,0 +1,55 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
<!--
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.
-->
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-kafka-bundle</artifactId>
<version>1.19.0-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>nifi-kafka-shared</artifactId>
<packaging>jar</packaging>
<dependencies>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-utils</artifactId>
<version>1.19.0-SNAPSHOT</version>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-ssl-context-service-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-kerberos-credentials-service-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-kerberos-user-service-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-security-kerberos-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-mock</artifactId>
</dependency>
</dependencies>
</project>

View File

@ -14,28 +14,29 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
package org.apache.nifi.kafka.shared.attribute;
import org.junit.jupiter.api.Test;
/**
* Kafka FlowFile Attributes
*/
public interface KafkaFlowFileAttribute {
String KAFKA_KEY = "kafka.key";
import java.util.function.Supplier;
String KAFKA_TOPIC = "kafka.topic";
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
String KAFKA_PARTITION = "kafka.partition";
public class KafkaProcessorUtilsTest {
String KAFKA_OFFSET = "kafka.offset";
@Test
public void getTransactionalIdSupplierWithPrefix() {
Supplier<String> prefix = KafkaProcessorUtils.getTransactionalIdSupplier("prefix");
String id = prefix.get();
assertTrue(id.startsWith("prefix"));
assertEquals(42, id.length());
}
@Test
public void getTransactionalIdSupplierWithEmptyPrefix() {
Supplier<String> prefix = KafkaProcessorUtils.getTransactionalIdSupplier(null);
assertEquals(36, prefix.get().length() );
}
String KAFKA_MAX_OFFSET = "kafka.max.offset";
String KAFKA_LEADER_EPOCH = "kafka.leader.epoch";
String KAFKA_TIMESTAMP = "kafka.timestamp";
String KAFKA_COUNT = "kafka.count";
String KAFKA_CONSUMER_GROUP_ID = "kafka.consumer.id";
String KAFKA_CONSUMER_OFFSETS_COMMITTED = "kafka.consumer.offsets.committed";
}

View File

@ -0,0 +1,36 @@
/*
* 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.kafka.shared.attribute;
/**
* Transit URI Provider for Provenance Reporting
*/
public class StandardTransitUriProvider {
private static final String TRANSIT_URI = "%s://%s/%s";
/**
* Get Transit URI
*
* @param securityProtocol Kafka Security Protocol
* @param brokers One or more hostname and port combinations of Kafka Brokers
* @param topic Kafka Topic
* @return Transit URI
*/
public static String getTransitUri(final String securityProtocol, final String brokers, final String topic) {
return String.format(TRANSIT_URI, securityProtocol, brokers, topic);
}
}

View File

@ -0,0 +1,160 @@
/*
* 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.kafka.shared.component;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.resource.ResourceCardinality;
import org.apache.nifi.components.resource.ResourceType;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.kafka.shared.property.SaslMechanism;
import org.apache.nifi.kafka.shared.property.SecurityProtocol;
import org.apache.nifi.kerberos.KerberosCredentialsService;
import org.apache.nifi.kerberos.SelfContainedKerberosUserService;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.ssl.SSLContextService;
/**
* Kafka Client Component interface with common Property Descriptors
*/
public interface KafkaClientComponent {
PropertyDescriptor BOOTSTRAP_SERVERS = new PropertyDescriptor.Builder()
.name("bootstrap.servers")
.displayName("Kafka Brokers")
.description("Comma-separated list of Kafka Brokers in the format host:port")
.required(true)
.addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.defaultValue("localhost:9092")
.build();
PropertyDescriptor SECURITY_PROTOCOL = new PropertyDescriptor.Builder()
.name("security.protocol")
.displayName("Security Protocol")
.description("Security protocol used to communicate with brokers. Corresponds to Kafka Client security.protocol property")
.required(true)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.allowableValues(SecurityProtocol.values())
.defaultValue(SecurityProtocol.PLAINTEXT.name())
.build();
PropertyDescriptor SASL_MECHANISM = new PropertyDescriptor.Builder()
.name("sasl.mechanism")
.displayName("SASL Mechanism")
.description("SASL mechanism used for authentication. Corresponds to Kafka Client sasl.mechanism property")
.required(true)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.allowableValues(SaslMechanism.class)
.defaultValue(SaslMechanism.GSSAPI.getValue())
.build();
PropertyDescriptor SASL_USERNAME = new PropertyDescriptor.Builder()
.name("sasl.username")
.displayName("Username")
.description("Username provided with configured password when using PLAIN or SCRAM SASL Mechanisms")
.required(false)
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.dependsOn(
SASL_MECHANISM,
SaslMechanism.PLAIN.getValue(),
SaslMechanism.SCRAM_SHA_256.getValue(),
SaslMechanism.SCRAM_SHA_512.getValue()
)
.build();
PropertyDescriptor SASL_PASSWORD = new PropertyDescriptor.Builder()
.name("sasl.password")
.displayName("Password")
.description("Password provided with configured username when using PLAIN or SCRAM SASL Mechanisms")
.required(false)
.sensitive(true)
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.dependsOn(
SASL_MECHANISM,
SaslMechanism.PLAIN.getValue(),
SaslMechanism.SCRAM_SHA_256.getValue(),
SaslMechanism.SCRAM_SHA_512.getValue()
)
.build();
PropertyDescriptor TOKEN_AUTHENTICATION = new PropertyDescriptor.Builder()
.name("sasl.token.auth")
.displayName("Token Authentication")
.description("Enables or disables Token authentication when using SCRAM SASL Mechanisms")
.required(false)
.allowableValues(Boolean.TRUE.toString(), Boolean.FALSE.toString())
.defaultValue(Boolean.FALSE.toString())
.dependsOn(
SASL_MECHANISM,
SaslMechanism.SCRAM_SHA_256.getValue(),
SaslMechanism.SCRAM_SHA_512.getValue()
)
.build();
PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
.name("ssl.context.service")
.displayName("SSL Context Service")
.description("Service supporting SSL communication with Kafka brokers")
.required(false)
.identifiesControllerService(SSLContextService.class)
.build();
PropertyDescriptor KERBEROS_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 configuration")
.required(false)
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
PropertyDescriptor KERBEROS_PRINCIPAL = new PropertyDescriptor.Builder()
.name("sasl.kerberos.principal")
.displayName("Kerberos Principal")
.description("Principal used for authentication with Kerberos")
.required(false)
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
PropertyDescriptor KERBEROS_KEYTAB = new PropertyDescriptor.Builder()
.name("sasl.kerberos.keytab")
.displayName("Kerberos Keytab")
.description("Keytab credentials used for authentication with Kerberos")
.required(false)
.identifiesExternalResource(ResourceCardinality.SINGLE, ResourceType.FILE)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
PropertyDescriptor KERBEROS_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
.name("kerberos-credentials-service")
.displayName("Kerberos Credentials Service")
.description("Service supporting generalized credentials authentication with Kerberos")
.identifiesControllerService(KerberosCredentialsService.class)
.required(false)
.build();
PropertyDescriptor SELF_CONTAINED_KERBEROS_USER_SERVICE = new PropertyDescriptor.Builder()
.name("kerberos-user-service")
.displayName("Kerberos User Service")
.description("Service supporting user authentication with Kerberos")
.identifiesControllerService(SelfContainedKerberosUserService.class)
.required(false)
.build();
}

View File

@ -0,0 +1,34 @@
/*
* 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.kafka.shared.component;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.kafka.shared.property.FailureStrategy;
/**
* Kafka Publish Component interface with common Property Descriptors
*/
public interface KafkaPublishComponent extends KafkaClientComponent {
PropertyDescriptor FAILURE_STRATEGY = new PropertyDescriptor.Builder()
.name("Failure Strategy")
.displayName("Failure Strategy")
.description("Specifies how the processor handles a FlowFile if it is unable to publish the data to Kafka")
.required(true)
.allowableValues(FailureStrategy.class)
.defaultValue(FailureStrategy.ROUTE_TO_FAILURE.getValue())
.build();
}

View File

@ -0,0 +1,54 @@
/*
* 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.kafka.shared.login;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.kafka.shared.component.KafkaClientComponent;
import org.apache.nifi.kafka.shared.property.SaslMechanism;
import java.util.LinkedHashMap;
import java.util.Map;
/**
* Delegating Login Module implementation of configuration provider
*/
public class DelegatingLoginConfigProvider implements LoginConfigProvider {
private static final LoginConfigProvider SCRAM_PROVIDER = new ScramLoginConfigProvider();
private static final Map<SaslMechanism, LoginConfigProvider> PROVIDERS = new LinkedHashMap<>();
static {
PROVIDERS.put(SaslMechanism.GSSAPI, new KerberosDelegatingLoginConfigProvider());
PROVIDERS.put(SaslMechanism.PLAIN, new PlainLoginConfigProvider());
PROVIDERS.put(SaslMechanism.SCRAM_SHA_256, SCRAM_PROVIDER);
PROVIDERS.put(SaslMechanism.SCRAM_SHA_512, SCRAM_PROVIDER);
}
/**
* Get JAAS configuration using configured username and password properties
*
* @param context Property Context
* @return JAAS configuration with Plain Login Module
*/
@Override
public String getConfiguration(final PropertyContext context) {
final String saslMechanismProperty = context.getProperty(KafkaClientComponent.SASL_MECHANISM).getValue();
final SaslMechanism saslMechanism = SaslMechanism.valueOf(saslMechanismProperty);
final LoginConfigProvider loginConfigProvider = PROVIDERS.getOrDefault(saslMechanism, SCRAM_PROVIDER);
return loginConfigProvider.getConfiguration(context);
}
}

View File

@ -0,0 +1,54 @@
/*
* 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.kafka.shared.login;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.kafka.shared.component.KafkaClientComponent;
import org.apache.nifi.kerberos.KerberosCredentialsService;
/**
* Kerberos Login Module implementation of configuration provider
*/
public class KerberosCredentialsLoginConfigProvider implements LoginConfigProvider {
private static final String MODULE_CLASS_NAME = "com.sun.security.auth.module.Krb5LoginModule";
private static final String FORMAT = "%s required renewTicket=true useKeyTab=true serviceName=\"%s\" principal=\"%s\" keyTab=\"%s\";";
/**
* Get JAAS configuration using configured Kerberos credentials
*
* @param context Property Context
* @return JAAS configuration with Kerberos Login Module
*/
@Override
public String getConfiguration(final PropertyContext context) {
final String principal;
final String keyTab;
final KerberosCredentialsService credentialsService = context.getProperty(KafkaClientComponent.KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
if (credentialsService == null) {
principal = context.getProperty(KafkaClientComponent.KERBEROS_PRINCIPAL).evaluateAttributeExpressions().getValue();
keyTab = context.getProperty(KafkaClientComponent.KERBEROS_KEYTAB).evaluateAttributeExpressions().getValue();
} else {
principal = credentialsService.getPrincipal();
keyTab = credentialsService.getKeytab();
}
final String serviceName = context.getProperty(KafkaClientComponent.KERBEROS_SERVICE_NAME).evaluateAttributeExpressions().getValue();
return String.format(FORMAT, MODULE_CLASS_NAME, serviceName, principal, keyTab);
}
}

View File

@ -0,0 +1,50 @@
/*
* 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.kafka.shared.login;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.kafka.shared.component.KafkaClientComponent;
/**
* Kerberos Delegating Login Module implementation of configuration provider
*/
public class KerberosDelegatingLoginConfigProvider implements LoginConfigProvider {
private static final LoginConfigProvider CREDENTIALS_PROVIDER = new KerberosCredentialsLoginConfigProvider();
private static final LoginConfigProvider USER_SERVICE_PROVIDER = new KerberosUserServiceLoginConfigProvider();
/**
* Get JAAS configuration using configured Kerberos credentials
*
* @param context Property Context
* @return JAAS configuration with Kerberos Login Module
*/
@Override
public String getConfiguration(final PropertyContext context) {
final PropertyValue userServiceProperty = context.getProperty(KafkaClientComponent.SELF_CONTAINED_KERBEROS_USER_SERVICE);
final String configuration;
if (userServiceProperty.isSet()) {
configuration = USER_SERVICE_PROVIDER.getConfiguration(context);
} else {
configuration = CREDENTIALS_PROVIDER.getConfiguration(context);
}
return configuration;
}
}

View File

@ -0,0 +1,92 @@
/*
* 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.kafka.shared.login;
import static org.apache.nifi.kafka.shared.component.KafkaClientComponent.SELF_CONTAINED_KERBEROS_USER_SERVICE;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.kerberos.KerberosUserService;
import org.apache.nifi.kerberos.SelfContainedKerberosUserService;
import org.apache.nifi.security.krb.KerberosUser;
import javax.security.auth.login.AppConfigurationEntry;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Objects;
/**
* Kerberos User Service Login Module implementation of configuration provider
*/
public class KerberosUserServiceLoginConfigProvider implements LoginConfigProvider {
private static final String SPACE = " ";
private static final String EQUALS = "=";
private static final String DOUBLE_QUOTE = "\"";
private static final String SEMI_COLON = ";";
private static final Map<AppConfigurationEntry.LoginModuleControlFlag, String> CONTROL_FLAGS = new LinkedHashMap<>();
static {
CONTROL_FLAGS.put(AppConfigurationEntry.LoginModuleControlFlag.OPTIONAL, "optional");
CONTROL_FLAGS.put(AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, "required");
CONTROL_FLAGS.put(AppConfigurationEntry.LoginModuleControlFlag.REQUISITE, "requisite");
CONTROL_FLAGS.put(AppConfigurationEntry.LoginModuleControlFlag.SUFFICIENT, "sufficient");
}
/**
* Get JAAS configuration using configured Kerberos credentials
*
* @param context Property Context
* @return JAAS configuration with Login Module based on User Service configuration
*/
@Override
public String getConfiguration(final PropertyContext context) {
final KerberosUserService kerberosUserService = context.getProperty(SELF_CONTAINED_KERBEROS_USER_SERVICE).asControllerService(SelfContainedKerberosUserService.class);
final KerberosUser kerberosUser = kerberosUserService.createKerberosUser();
final AppConfigurationEntry configurationEntry = kerberosUser.getConfigurationEntry();
final StringBuilder builder = new StringBuilder();
final String loginModuleName = configurationEntry.getLoginModuleName();
builder.append(loginModuleName);
final AppConfigurationEntry.LoginModuleControlFlag controlFlag = configurationEntry.getControlFlag();
final String moduleControlFlag = Objects.requireNonNull(CONTROL_FLAGS.get(controlFlag), "Control Flag not found");
builder.append(SPACE);
builder.append(moduleControlFlag);
final Map<String, ?> options = configurationEntry.getOptions();
options.forEach((key, value) -> {
builder.append(SPACE);
builder.append(key);
builder.append(EQUALS);
if (value instanceof String) {
builder.append(DOUBLE_QUOTE);
builder.append(value);
builder.append(DOUBLE_QUOTE);
} else {
builder.append(value);
}
});
builder.append(SEMI_COLON);
return builder.toString();
}
}

View File

@ -0,0 +1,32 @@
/*
* 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.kafka.shared.login;
import org.apache.nifi.context.PropertyContext;
/**
* Provider abstraction for Kafka SASL JAAS configuration
*/
public interface LoginConfigProvider {
/**
* Get SASL JAAS configuration using configured properties
*
* @param context Property Context
* @return JAAS configuration
*/
String getConfiguration(PropertyContext context);
}

View File

@ -0,0 +1,42 @@
/*
* 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.kafka.shared.login;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.kafka.shared.component.KafkaClientComponent;
/**
* SASL Plain Login Module implementation of configuration provider
*/
public class PlainLoginConfigProvider implements LoginConfigProvider {
private static final String MODULE_CLASS_NAME = "org.apache.kafka.common.security.plain.PlainLoginModule";
private static final String FORMAT = "%s required username=\"%s\" password=\"%s\";";
/**
* Get JAAS configuration using configured username and password properties
*
* @param context Property Context
* @return JAAS configuration with Plain Login Module
*/
@Override
public String getConfiguration(final PropertyContext context) {
final String username = context.getProperty(KafkaClientComponent.SASL_USERNAME).evaluateAttributeExpressions().getValue();
final String password = context.getProperty(KafkaClientComponent.SASL_PASSWORD).evaluateAttributeExpressions().getValue();
return String.format(FORMAT, MODULE_CLASS_NAME, username, password);
}
}

View File

@ -0,0 +1,58 @@
/*
* 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.kafka.shared.login;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.kafka.shared.component.KafkaClientComponent;
/**
* SASL SCRAM Login Module implementation of configuration provider
*/
public class ScramLoginConfigProvider implements LoginConfigProvider {
private static final String MODULE_CLASS_NAME = "org.apache.kafka.common.security.scram.ScramLoginModule";
private static final String FORMAT = "%s required username=\"%s\" password=\"%s\"";
private static final String TOKEN_AUTH_ENABLED = "tokenauth=true";
private static final String SEMI_COLON = ";";
/**
* Get JAAS configuration using configured username and password with optional token authentication
*
* @param context Property Context
* @return JAAS configuration with SCRAM Login Module
*/
@Override
public String getConfiguration(final PropertyContext context) {
final StringBuilder builder = new StringBuilder();
final String username = context.getProperty(KafkaClientComponent.SASL_USERNAME).evaluateAttributeExpressions().getValue();
final String password = context.getProperty(KafkaClientComponent.SASL_PASSWORD).evaluateAttributeExpressions().getValue();
final String moduleUsernamePassword = String.format(FORMAT, MODULE_CLASS_NAME, username, password);
builder.append(moduleUsernamePassword);
final Boolean tokenAuthenticationEnabled = context.getProperty(KafkaClientComponent.TOKEN_AUTHENTICATION).asBoolean();
if (Boolean.TRUE == tokenAuthenticationEnabled) {
builder.append(TOKEN_AUTH_ENABLED);
}
builder.append(SEMI_COLON);
return builder.toString();
}
}

View File

@ -0,0 +1,56 @@
/*
* 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.kafka.shared.property;
import org.apache.nifi.components.DescribedValue;
/**
* Enumeration of supported Kafka Publishing Failure Strategies
*/
public enum FailureStrategy implements DescribedValue {
ROUTE_TO_FAILURE("Route to Failure", "Route to Failure", "When unable to publish records to Kafka, the FlowFile will be routed to the failure relationship."),
ROLLBACK("Rollback", "Rollback", "When unable to publish records to Kafka, the FlowFile will be placed back on the queue so that it will be retried. " +
"For flows where FlowFile ordering is important, this strategy can be used along with ensuring that the each processor uses only a single Concurrent Task.");
private final String value;
private final String displayName;
private final String description;
FailureStrategy(final String value, final String displayName, final String description) {
this.value = value;
this.displayName = displayName;
this.description = description;
}
@Override
public String getValue() {
return value;
}
@Override
public String getDisplayName() {
return displayName;
}
@Override
public String getDescription() {
return description;
}
}

View File

@ -0,0 +1,50 @@
/*
* 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.kafka.shared.property;
/**
* Enumeration of Kafka Client property names without reference to Kafka libraries
*/
public enum KafkaClientProperty {
SASL_JAAS_CONFIG("sasl.jaas.config"),
SASL_LOGIN_CLASS("sasl.login.class"),
SSL_KEYSTORE_LOCATION("ssl.keystore.location"),
SSL_KEYSTORE_PASSWORD("ssl.keystore.password"),
SSL_KEYSTORE_TYPE("ssl.keystore.type"),
SSL_KEY_PASSWORD("ssl.key.password"),
SSL_TRUSTSTORE_LOCATION("ssl.truststore.location"),
SSL_TRUSTSTORE_PASSWORD("ssl.truststore.password"),
SSL_TRUSTSTORE_TYPE("ssl.truststore.type");
private final String property;
KafkaClientProperty(final String property) {
this.property = property;
}
public String getProperty() {
return property;
}
}

View File

@ -0,0 +1,57 @@
/*
* 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.kafka.shared.property;
import org.apache.nifi.components.DescribedValue;
/**
* Enumeration of supported Kafka Key Encoding Strategies
*/
public enum KeyEncoding implements DescribedValue {
UTF8("utf-8", "UTF-8 Encoded", "The key is interpreted as a UTF-8 Encoded string."),
HEX("hex", "Hex Encoded", "The key is interpreted as arbitrary binary data and is encoded using hexadecimal characters with uppercase letters"),
DO_NOT_ADD("do-not-add", "Do Not Add Key as Attribute","The key will not be added as an Attribute");
private final String value;
private final String displayName;
private final String description;
KeyEncoding(final String value, final String displayName, final String description) {
this.value = value;
this.displayName = displayName;
this.description = description;
}
@Override
public String getValue() {
return value;
}
@Override
public String getDisplayName() {
return displayName;
}
@Override
public String getDescription() {
return description;
}
}

View File

@ -0,0 +1,57 @@
/*
* 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.kafka.shared.property;
import org.apache.nifi.components.DescribedValue;
/**
* Enumeration of supported Kafka Key Formatting Strategies
*/
public enum KeyFormat implements DescribedValue {
STRING("string", "String", "Format the Kafka ConsumerRecord key as a UTF-8 string."),
BYTE_ARRAY("byte-array", "Byte Array", "Format the Kafka ConsumerRecord key as a byte array."),
RECORD("record", "Record", "Format the Kafka ConsumerRecord key as a record.");
private final String value;
private final String displayName;
private final String description;
KeyFormat(final String value, final String displayName, final String description) {
this.value = value;
this.displayName = displayName;
this.description = description;
}
@Override
public String getValue() {
return value;
}
@Override
public String getDisplayName() {
return displayName;
}
@Override
public String getDescription() {
return description;
}
}

View File

@ -0,0 +1,55 @@
/*
* 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.kafka.shared.property;
import org.apache.nifi.components.DescribedValue;
/**
* Enumeration of supported Kafka Output Strategies
*/
public enum OutputStrategy implements DescribedValue {
USE_VALUE("USE_VALUE", "Use Content as Value", "Write only the Kafka Record value to the FlowFile record."),
USE_WRAPPER("USE_WRAPPER", "Use Wrapper", "Write the Kafka Record key, value, headers, and metadata into the FlowFile record. (See processor usage for more information.)");
private final String value;
private final String displayName;
private final String description;
OutputStrategy(final String value, final String displayName, final String description) {
this.value = value;
this.displayName = displayName;
this.description = description;
}
@Override
public String getValue() {
return value;
}
@Override
public String getDisplayName() {
return displayName;
}
@Override
public String getDescription() {
return description;
}
}

View File

@ -0,0 +1,52 @@
/*
* 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.kafka.shared.property;
import org.apache.nifi.components.DescribedValue;
/**
* Enumeration of supported Kafka Publishing Strategies
*/
public enum PublishStrategy implements DescribedValue {
USE_VALUE( "Use Content as Record Value", "Write only the FlowFile content to the Kafka Record value."),
USE_WRAPPER("Use Wrapper", "Write the Kafka Record key, value, headers, and metadata into the Kafka Record value. (See processor usage for more information.)");
private final String displayName;
private final String description;
PublishStrategy(final String displayName, final String description) {
this.displayName = displayName;
this.description = description;
}
@Override
public String getValue() {
return name();
}
@Override
public String getDisplayName() {
return displayName;
}
@Override
public String getDescription() {
return description;
}
}

View File

@ -0,0 +1,59 @@
/*
* 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.kafka.shared.property;
import org.apache.nifi.components.DescribedValue;
/**
* Enumeration of supported Kafka SASL Mechanisms
*/
public enum SaslMechanism implements DescribedValue {
GSSAPI("GSSAPI", "GSSAPI", "General Security Services API for Kerberos authentication"),
PLAIN("PLAIN", "PLAIN", "Plain username and password authentication"),
SCRAM_SHA_256("SCRAM-SHA-256", "SCRAM-SHA-256", "Salted Challenge Response Authentication Mechanism using SHA-512 with username and password"),
SCRAM_SHA_512("SCRAM-SHA-512", "SCRAM-SHA-512", "Salted Challenge Response Authentication Mechanism using SHA-256 with username and password");
private final String value;
private final String displayName;
private final String description;
SaslMechanism(final String value, final String displayName, final String description) {
this.value = value;
this.displayName = displayName;
this.description = description;
}
@Override
public String getValue() {
return value;
}
@Override
public String getDisplayName() {
return displayName;
}
@Override
public String getDescription() {
return description;
}
}

View File

@ -14,12 +14,17 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
package org.apache.nifi.kafka.shared.property;
/**
* Enumeration of strategies used by {@link ConsumeKafkaRecord_2_6} to map Kafka records to NiFi FlowFiles.
* Enumeration of supported Kafka Security Protocols
*/
public enum OutputStrategy {
USE_VALUE,
USE_WRAPPER;
public enum SecurityProtocol {
PLAINTEXT,
SSL,
SASL_PLAINTEXT,
SASL_SSL
}

View File

@ -14,12 +14,18 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
package org.apache.nifi.kafka.shared.property.provider;
import java.util.Set;
/**
* Enumeration of strategies used by {@link PublishKafkaRecord_2_6} to map NiFi FlowFiles to Kafka records.
* Provider for Kafka Client Property Names
*/
public enum PublishStrategy {
USE_VALUE,
USE_WRAPPER;
public interface KafkaPropertyNameProvider {
/**
* Get Property Names
*
* @return Kafka Client Property Names
*/
Set<String> getPropertyNames();
}

View File

@ -0,0 +1,34 @@
/*
* 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.kafka.shared.property.provider;
import org.apache.nifi.context.PropertyContext;
import java.util.Map;
/**
* Provider abstraction for configuring Kafka Consumer and Producer properties
*/
public interface KafkaPropertyProvider {
/**
* Get Kafka Properties
*
* @param context Property Context
* @return Kafka Properties
*/
Map<String, Object> getProperties(PropertyContext context);
}

View File

@ -0,0 +1,103 @@
/*
* 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.kafka.shared.property.provider;
import java.lang.reflect.Field;
import java.lang.reflect.Modifier;
import java.util.LinkedHashSet;
import java.util.Set;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
/**
* Standard implementation of Kafka Property Name Provider using class name references from Kafka library
*/
public class StandardKafkaPropertyNameProvider implements KafkaPropertyNameProvider {
private static final String COMMON_CLIENT_CONFIGS_CLASS = "org.apache.kafka.clients.CommonClientConfigs";
private static final String SASL_CONFIGS_CLASS = "org.apache.kafka.common.config.SaslConfigs";
private static final String SSL_CONFIGS_CLASS = "org.apache.kafka.common.config.SslConfigs";
private static final String[] PROPERTY_CLASSES = new String[]{
COMMON_CLIENT_CONFIGS_CLASS,
SASL_CONFIGS_CLASS,
SSL_CONFIGS_CLASS
};
private static final Pattern PROPERTY_PATTERN = Pattern.compile("^\\S+$");
private final Set<String> propertyNames;
public StandardKafkaPropertyNameProvider(final Class<?> kafkaClientClass) {
final Set<String> kafkaClientPropertyNames = getStaticStringPropertyNames(kafkaClientClass);
kafkaClientPropertyNames.addAll(getCommonPropertyNames());
propertyNames = kafkaClientPropertyNames;
}
@Override
public Set<String> getPropertyNames() {
return propertyNames;
}
private static Set<String> getCommonPropertyNames() {
final Set<String> propertyNames = new LinkedHashSet<>();
for (final String propertyClassName : PROPERTY_CLASSES) {
final Class<?> propertyClass = getClass(propertyClassName);
final Set<String> classPropertyNames = getStaticStringPropertyNames(propertyClass);
propertyNames.addAll(classPropertyNames);
}
return propertyNames;
}
private static Set<String> getStaticStringPropertyNames(final Class<?> propertyClass) {
final Set<String> propertyNames = new LinkedHashSet<>();
for (final Field field : propertyClass.getDeclaredFields()) {
final int modifiers = field.getModifiers();
final Class<?> fieldType = field.getType();
if (Modifier.isPublic(modifiers) && Modifier.isStatic(modifiers) && String.class.equals(fieldType)) {
final String fieldValue = getStaticFieldValue(field);
final Matcher propertyMatcher = PROPERTY_PATTERN.matcher(fieldValue);
if (propertyMatcher.matches()) {
propertyNames.add(fieldValue);
}
}
}
return propertyNames;
}
private static String getStaticFieldValue(final Field field) {
try {
return String.valueOf(field.get(null));
} catch (final Exception e) {
final String message = String.format("Unable to read Kafka Configuration class field [%s]", field.getName());
throw new IllegalArgumentException(message, e);
}
}
private static Class<?> getClass(final String className) {
try {
return Class.forName(className);
} catch (final ClassNotFoundException e) {
throw new IllegalStateException("Kafka Configuration Class not found", e);
}
}
}

View File

@ -0,0 +1,171 @@
/*
* 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.kafka.shared.property.provider;
import static org.apache.nifi.kafka.shared.component.KafkaClientComponent.SASL_MECHANISM;
import static org.apache.nifi.kafka.shared.component.KafkaClientComponent.SECURITY_PROTOCOL;
import static org.apache.nifi.kafka.shared.component.KafkaClientComponent.SSL_CONTEXT_SERVICE;
import static org.apache.nifi.kafka.shared.property.KafkaClientProperty.SASL_JAAS_CONFIG;
import static org.apache.nifi.kafka.shared.property.KafkaClientProperty.SASL_LOGIN_CLASS;
import static org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_KEYSTORE_LOCATION;
import static org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_KEYSTORE_PASSWORD;
import static org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_KEYSTORE_TYPE;
import static org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_KEY_PASSWORD;
import static org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_TRUSTSTORE_LOCATION;
import static org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_TRUSTSTORE_PASSWORD;
import static org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_TRUSTSTORE_TYPE;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.kafka.shared.login.DelegatingLoginConfigProvider;
import org.apache.nifi.kafka.shared.login.LoginConfigProvider;
import org.apache.nifi.kafka.shared.property.SaslMechanism;
import org.apache.nifi.kafka.shared.property.SecurityProtocol;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.ssl.SSLContextService;
import org.apache.nifi.util.FormatUtils;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.regex.Matcher;
import java.util.stream.Collectors;
/**
* Standard implementation of Kafka Property Provider based on shared Kafka Property Descriptors
*/
public class StandardKafkaPropertyProvider implements KafkaPropertyProvider {
private static final String MILLISECOND_PROPERTY_SUFFIX = ".ms";
private static final String SASL_GSSAPI_CUSTOM_LOGIN_CLASS = "org.apache.nifi.processors.kafka.pubsub.CustomKerberosLogin";
private static final LoginConfigProvider LOGIN_CONFIG_PROVIDER = new DelegatingLoginConfigProvider();
private final Set<String> clientPropertyNames;
public StandardKafkaPropertyProvider(final Class<?> kafkaClientClass) {
final KafkaPropertyNameProvider provider = new StandardKafkaPropertyNameProvider(kafkaClientClass);
clientPropertyNames = provider.getPropertyNames();
}
@Override
public Map<String, Object> getProperties(final PropertyContext context) {
final Map<String, Object> properties = new LinkedHashMap<>();
setClientProperties(properties, context);
setSecurityProperties(properties, context);
setSslProperties(properties, context);
return properties;
}
private void setSecurityProperties(final Map<String, Object> properties, final PropertyContext context) {
final String protocol = context.getProperty(SECURITY_PROTOCOL).getValue();
properties.put(SECURITY_PROTOCOL.getName(), protocol);
final SecurityProtocol securityProtocol = SecurityProtocol.valueOf(protocol);
if (SecurityProtocol.SASL_PLAINTEXT == securityProtocol || SecurityProtocol.SASL_SSL == securityProtocol) {
final String loginConfig = LOGIN_CONFIG_PROVIDER.getConfiguration(context);
properties.put(SASL_JAAS_CONFIG.getProperty(), loginConfig);
final SaslMechanism saslMechanism = SaslMechanism.valueOf(context.getProperty(SASL_MECHANISM).getValue());
if (SaslMechanism.GSSAPI == saslMechanism && isCustomKerberosLoginFound()) {
properties.put(SASL_LOGIN_CLASS.getProperty(), SASL_GSSAPI_CUSTOM_LOGIN_CLASS);
}
}
}
private void setSslProperties(final Map<String, Object> properties, final PropertyContext context) {
final PropertyValue sslContextServiceProperty = context.getProperty(SSL_CONTEXT_SERVICE);
if (sslContextServiceProperty.isSet()) {
final SSLContextService sslContextService = sslContextServiceProperty.asControllerService(SSLContextService.class);
if (sslContextService.isKeyStoreConfigured()) {
properties.put(SSL_KEYSTORE_LOCATION.getProperty(), sslContextService.getKeyStoreFile());
properties.put(SSL_KEYSTORE_TYPE.getProperty(), sslContextService.getKeyStoreType());
final String keyStorePassword = sslContextService.getKeyStorePassword();
properties.put(SSL_KEYSTORE_PASSWORD.getProperty(), keyStorePassword);
final String keyPassword = sslContextService.getKeyPassword();
final String configuredKeyPassword = keyPassword == null ? keyStorePassword : keyPassword;
properties.put(SSL_KEY_PASSWORD.getProperty(), configuredKeyPassword);
}
if (sslContextService.isTrustStoreConfigured()) {
properties.put(SSL_TRUSTSTORE_LOCATION.getProperty(), sslContextService.getTrustStoreFile());
properties.put(SSL_TRUSTSTORE_TYPE.getProperty(), sslContextService.getTrustStoreType());
properties.put(SSL_TRUSTSTORE_PASSWORD.getProperty(), sslContextService.getTrustStorePassword());
}
}
}
private void setClientProperties(final Map<String, Object> properties, final PropertyContext context) {
final Set<PropertyDescriptor> propertyDescriptors = getPropertyDescriptors(context).stream()
.filter(propertyDescriptor -> clientPropertyNames.contains(propertyDescriptor.getName()))
.collect(Collectors.toSet());
for (final PropertyDescriptor propertyDescriptor : propertyDescriptors) {
final PropertyValue property = context.getProperty(propertyDescriptor);
final String propertyValue = propertyDescriptor.isExpressionLanguageSupported()
? property.evaluateAttributeExpressions().getValue()
: property.getValue();
if (propertyValue == null) {
continue;
}
final String propertyName = propertyDescriptor.getName();
setProperty(properties, propertyName, propertyValue);
}
}
private Set<PropertyDescriptor> getPropertyDescriptors(final PropertyContext context) {
final Set<PropertyDescriptor> propertyDescriptors;
if (context instanceof ConfigurationContext) {
final ConfigurationContext configurationContext = (ConfigurationContext) context;
propertyDescriptors = configurationContext.getProperties().keySet();
} else if (context instanceof ProcessContext) {
final ProcessContext processContext = (ProcessContext) context;
propertyDescriptors = processContext.getProperties().keySet();
} else {
throw new IllegalArgumentException(String.format("Property Context [%s] not supported", context.getClass().getName()));
}
return propertyDescriptors;
}
private void setProperty(final Map<String, Object> properties, final String propertyName, final String propertyValue) {
if (propertyName.endsWith(MILLISECOND_PROPERTY_SUFFIX)) {
final Matcher durationMatcher = FormatUtils.TIME_DURATION_PATTERN.matcher(propertyValue);
if (durationMatcher.matches()) {
final long milliseconds = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
properties.put(propertyName, Long.toString(milliseconds));
} else {
properties.put(propertyName, propertyValue);
}
} else {
properties.put(propertyName, propertyValue);
}
}
private boolean isCustomKerberosLoginFound() {
try {
Class.forName(SASL_GSSAPI_CUSTOM_LOGIN_CLASS);
return true;
} catch (final ClassNotFoundException e) {
return false;
}
}
}

View File

@ -14,28 +14,30 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import org.junit.jupiter.api.Test;
package org.apache.nifi.kafka.shared.transaction;
import java.util.UUID;
import java.util.function.Supplier;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Standard Transaction Identifier Supplier with optional prefix
*/
public class TransactionIdSupplier implements Supplier<String> {
private static final String EMPTY_PREFIX = "";
public class KafkaProcessorUtilsTest {
private final String prefix;
@Test
public void getTransactionalIdSupplierWithPrefix() {
Supplier<String> prefix = KafkaProcessorUtils.getTransactionalIdSupplier("prefix");
String id = prefix.get();
assertTrue(id.startsWith("prefix"));
assertEquals(42, id.length());
public TransactionIdSupplier(final String prefix) {
this.prefix = prefix == null ? EMPTY_PREFIX : prefix;
}
@Test
public void getTransactionalIdSupplierWithEmptyPrefix() {
Supplier<String> prefix = KafkaProcessorUtils.getTransactionalIdSupplier(null);
assertEquals(36, prefix.get().length() );
/**
* Get Transaction Identifier consisting of a random UUID with configured prefix string
*
* @return Transaction Identifier
*/
@Override
public String get() {
return prefix + UUID.randomUUID();
}
}

View File

@ -0,0 +1,55 @@
/*
* 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.kafka.shared.validation;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator;
import org.apache.nifi.kafka.shared.property.provider.KafkaPropertyNameProvider;
import org.apache.nifi.kafka.shared.property.provider.StandardKafkaPropertyNameProvider;
import java.util.Set;
/**
* Validator for dynamic Kafka properties
*/
public class DynamicPropertyValidator implements Validator {
private static final String PARTITIONS_PROPERTY_PREFIX = "partitions";
private final Set<String> clientPropertyNames;
public DynamicPropertyValidator(final Class<?> kafkaClientClass) {
final KafkaPropertyNameProvider provider = new StandardKafkaPropertyNameProvider(kafkaClientClass);
clientPropertyNames = provider.getPropertyNames();
}
@Override
public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
final ValidationResult.Builder builder = new ValidationResult.Builder();
builder.subject(subject);
if (subject.startsWith(PARTITIONS_PROPERTY_PREFIX)) {
builder.valid(true);
} else {
final boolean valid = clientPropertyNames.contains(subject);
builder.valid(valid);
builder.explanation("must be a known Kafka client configuration property");
}
return builder.build();
}
}

View File

@ -0,0 +1,243 @@
/*
* 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.kafka.shared.validation;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.kafka.shared.property.KafkaClientProperty;
import org.apache.nifi.kafka.shared.property.SaslMechanism;
import org.apache.nifi.kafka.shared.property.SecurityProtocol;
import org.apache.nifi.kerberos.KerberosCredentialsService;
import org.apache.nifi.kerberos.KerberosUserService;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Optional;
import java.util.function.Function;
import static org.apache.nifi.kafka.shared.component.KafkaClientComponent.KERBEROS_CREDENTIALS_SERVICE;
import static org.apache.nifi.kafka.shared.component.KafkaClientComponent.KERBEROS_KEYTAB;
import static org.apache.nifi.kafka.shared.component.KafkaClientComponent.KERBEROS_PRINCIPAL;
import static org.apache.nifi.kafka.shared.component.KafkaClientComponent.KERBEROS_SERVICE_NAME;
import static org.apache.nifi.kafka.shared.component.KafkaClientComponent.SASL_MECHANISM;
import static org.apache.nifi.kafka.shared.component.KafkaClientComponent.SASL_PASSWORD;
import static org.apache.nifi.kafka.shared.component.KafkaClientComponent.SASL_USERNAME;
import static org.apache.nifi.kafka.shared.component.KafkaClientComponent.SECURITY_PROTOCOL;
import static org.apache.nifi.kafka.shared.component.KafkaClientComponent.SELF_CONTAINED_KERBEROS_USER_SERVICE;
/**
* Custom Validation function for components supporting Kafka clients
*/
public class KafkaClientCustomValidationFunction implements Function<ValidationContext, Collection<ValidationResult>> {
static final String JAVA_SECURITY_AUTH_LOGIN_CONFIG = "java.security.auth.login.config";
private static final String ALLOW_EXPLICIT_KEYTAB = "NIFI_ALLOW_EXPLICIT_KEYTAB";
private static final String JNDI_LOGIN_MODULE_CLASS = "JndiLoginModule";
private static final String JND_LOGIN_MODULE_EXPLANATION = "The JndiLoginModule is not allowed in the JAAS configuration";
private static final List<String> USERNAME_PASSWORD_SASL_MECHANISMS = Arrays.asList(
SaslMechanism.PLAIN.getValue(),
SaslMechanism.SCRAM_SHA_256.getValue(),
SaslMechanism.SCRAM_SHA_512.getValue()
);
private static final List<String> SASL_PROTOCOLS = Arrays.asList(
SecurityProtocol.SASL_PLAINTEXT.name(),
SecurityProtocol.SASL_SSL.name()
);
@Override
public Collection<ValidationResult> apply(final ValidationContext validationContext) {
final Collection<ValidationResult> results = new ArrayList<>();
validateLoginModule(validationContext, results);
validateKerberosServices(validationContext, results);
validateKerberosCredentials(validationContext, results);
validateUsernamePassword(validationContext, results);
return results;
}
private void validateLoginModule(final ValidationContext validationContext, final Collection<ValidationResult> results) {
final Optional<PropertyDescriptor> propertyDescriptorFound = validationContext.getProperties()
.keySet()
.stream()
.filter(
propertyDescriptor -> KafkaClientProperty.SASL_JAAS_CONFIG.getProperty().equals(propertyDescriptor.getName())
)
.findFirst();
if (propertyDescriptorFound.isPresent()) {
final PropertyDescriptor propertyDescriptor = propertyDescriptorFound.get();
final String saslJaasConfig = validationContext.getProperty(propertyDescriptor).getValue();
if (saslJaasConfig.contains(JNDI_LOGIN_MODULE_CLASS)) {
results.add(new ValidationResult.Builder()
.subject(propertyDescriptor.getName())
.valid(false)
.explanation(JND_LOGIN_MODULE_EXPLANATION)
.build());
}
}
}
private void validateKerberosServices(final ValidationContext validationContext, final Collection<ValidationResult> results) {
final PropertyValue userServiceProperty = validationContext.getProperty(SELF_CONTAINED_KERBEROS_USER_SERVICE);
final PropertyValue credentialsServiceProperty = validationContext.getProperty(KERBEROS_CREDENTIALS_SERVICE);
final String principal = validationContext.getProperty(KERBEROS_PRINCIPAL).evaluateAttributeExpressions().getValue();
final String keyTab = validationContext.getProperty(KERBEROS_KEYTAB).evaluateAttributeExpressions().getValue();
if (userServiceProperty.isSet()) {
if (credentialsServiceProperty.isSet()) {
final String explanation = String.format("Cannot configure both [%s] and [%s]",
SELF_CONTAINED_KERBEROS_USER_SERVICE.getDisplayName(),
KERBEROS_CREDENTIALS_SERVICE.getDisplayName()
);
results.add(new ValidationResult.Builder()
.subject(KERBEROS_CREDENTIALS_SERVICE.getDisplayName())
.valid(false)
.explanation(explanation)
.build());
}
if (isNotEmpty(principal) || isNotEmpty(keyTab)) {
final String explanation = String.format("Cannot configure [%s] with [%s] or [%s]",
SELF_CONTAINED_KERBEROS_USER_SERVICE.getDisplayName(),
KERBEROS_PRINCIPAL.getDisplayName(),
KERBEROS_KEYTAB.getDisplayName()
);
results.add(new ValidationResult.Builder()
.subject(SELF_CONTAINED_KERBEROS_USER_SERVICE.getDisplayName())
.valid(false)
.explanation(explanation)
.build());
}
} else if (credentialsServiceProperty.isSet()) {
if (isNotEmpty(principal) || isNotEmpty(keyTab)) {
final String explanation = String.format("Cannot configure [%s] with [%s] or [%s]",
KERBEROS_CREDENTIALS_SERVICE.getDisplayName(),
KERBEROS_PRINCIPAL.getDisplayName(),
KERBEROS_KEYTAB.getDisplayName()
);
results.add(new ValidationResult.Builder()
.subject(KERBEROS_CREDENTIALS_SERVICE.getDisplayName())
.valid(false)
.explanation(explanation)
.build());
}
}
final String allowExplicitKeytab = System.getenv(ALLOW_EXPLICIT_KEYTAB);
if (Boolean.FALSE.toString().equalsIgnoreCase(allowExplicitKeytab) && (isNotEmpty(principal) || isNotEmpty(keyTab))) {
final String explanation = String.format("Environment Variable [%s] disables configuring [%s] and [%s] properties",
ALLOW_EXPLICIT_KEYTAB,
KERBEROS_PRINCIPAL.getDisplayName(),
KERBEROS_KEYTAB.getDisplayName()
);
results.add(new ValidationResult.Builder()
.subject(KERBEROS_PRINCIPAL.getDisplayName())
.valid(false)
.explanation(explanation)
.build());
}
}
private void validateKerberosCredentials(final ValidationContext validationContext, final Collection<ValidationResult> results) {
final String saslMechanism = validationContext.getProperty(SASL_MECHANISM).getValue();
final String securityProtocol = validationContext.getProperty(SECURITY_PROTOCOL).getValue();
if (SaslMechanism.GSSAPI.name().equals(saslMechanism) && SASL_PROTOCOLS.contains(securityProtocol)) {
final String serviceName = validationContext.getProperty(KERBEROS_SERVICE_NAME).evaluateAttributeExpressions().getValue();
if (isEmpty(serviceName)) {
final String explanation = String.format("[%s] required for [%s] value [%s]", KERBEROS_SERVICE_NAME.getDisplayName(), SASL_MECHANISM.getDisplayName(), SaslMechanism.GSSAPI);
results.add(new ValidationResult.Builder()
.subject(KERBEROS_SERVICE_NAME.getDisplayName())
.valid(false)
.explanation(explanation)
.build());
}
final String principal = validationContext.getProperty(KERBEROS_PRINCIPAL).evaluateAttributeExpressions().getValue();
final String keyTab = validationContext.getProperty(KERBEROS_KEYTAB).evaluateAttributeExpressions().getValue();
final String systemLoginConfig = System.getProperty(JAVA_SECURITY_AUTH_LOGIN_CONFIG);
if (isEmpty(principal) && isNotEmpty(keyTab)) {
final String explanation = String.format("[%s] required when configuring [%s]", KERBEROS_KEYTAB.getDisplayName(), KERBEROS_PRINCIPAL.getDisplayName());
results.add(new ValidationResult.Builder()
.subject(KERBEROS_PRINCIPAL.getDisplayName())
.valid(false)
.explanation(explanation)
.build());
} else if (isNotEmpty(principal) && isEmpty(keyTab)) {
final String explanation = String.format("[%s] required when configuring [%s]", KERBEROS_PRINCIPAL.getDisplayName(), KERBEROS_KEYTAB.getDisplayName());
results.add(new ValidationResult.Builder()
.subject(KERBEROS_KEYTAB.getDisplayName())
.valid(false)
.explanation(explanation)
.build());
}
final KerberosUserService userService = validationContext.getProperty(SELF_CONTAINED_KERBEROS_USER_SERVICE).asControllerService(KerberosUserService.class);
final KerberosCredentialsService credentialsService = validationContext.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
if (userService == null && credentialsService == null && isEmpty(principal) && isEmpty(keyTab) && isEmpty(systemLoginConfig)) {
final String explanation = String.format("Kerberos Credentials not found in component properties or System Property [%s]", JAVA_SECURITY_AUTH_LOGIN_CONFIG);
results.add(new ValidationResult.Builder()
.subject(SASL_MECHANISM.getDisplayName())
.valid(false)
.explanation(explanation)
.build());
}
}
}
private void validateUsernamePassword(final ValidationContext validationContext, final Collection<ValidationResult> results) {
final String saslMechanism = validationContext.getProperty(SASL_MECHANISM).getValue();
if (USERNAME_PASSWORD_SASL_MECHANISMS.contains(saslMechanism)) {
final String username = validationContext.getProperty(SASL_USERNAME).evaluateAttributeExpressions().getValue();
if (username == null || username.isEmpty()) {
final String explanation = String.format("[%s] required for [%s] values: %s", SASL_USERNAME.getDisplayName(), SASL_MECHANISM.getDisplayName(), USERNAME_PASSWORD_SASL_MECHANISMS);
results.add(new ValidationResult.Builder()
.subject(SASL_USERNAME.getDisplayName())
.valid(false)
.explanation(explanation)
.build());
}
final String password = validationContext.getProperty(SASL_PASSWORD).evaluateAttributeExpressions().getValue();
if (password == null || password.isEmpty()) {
final String explanation = String.format("[%s] required for [%s] values: %s", SASL_PASSWORD.getDisplayName(), SASL_MECHANISM.getDisplayName(), USERNAME_PASSWORD_SASL_MECHANISMS);
results.add(new ValidationResult.Builder()
.subject(SASL_PASSWORD.getDisplayName())
.valid(false)
.explanation(explanation)
.build());
}
}
}
private boolean isEmpty(final String string) {
return string == null || string.isEmpty();
}
private boolean isNotEmpty(final String string) {
return string != null && !string.isEmpty();
}
}

View File

@ -0,0 +1,179 @@
/*
* 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.kafka.shared.validation;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.kafka.shared.component.KafkaClientComponent;
import org.apache.nifi.kafka.shared.property.KafkaClientProperty;
import org.apache.nifi.kafka.shared.property.SaslMechanism;
import org.apache.nifi.kafka.shared.property.SecurityProtocol;
import org.apache.nifi.util.MockProcessContext;
import org.apache.nifi.util.MockValidationContext;
import org.apache.nifi.util.NoOpProcessor;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.util.Collection;
import java.util.Optional;
import static org.junit.jupiter.api.Assertions.assertTrue;
class KafkaClientCustomValidationFunctionTest {
private static final String JAAS_CONFIG_JNDI_LOGIN_MODULE = "com.sun.security.auth.module.JndiLoginModule required debug=true;";
private static final String JAAS_CONFIG_PLACEHOLDER = "jaas.config";
TestRunner runner;
KafkaClientCustomValidationFunction validationFunction;
@BeforeEach
void setValidationFunction() {
System.clearProperty(KafkaClientCustomValidationFunction.JAVA_SECURITY_AUTH_LOGIN_CONFIG);
validationFunction = new KafkaClientCustomValidationFunction();
runner = TestRunners.newTestRunner(NoOpProcessor.class);
}
@Test
void testApply() {
final ValidationContext validationContext = getValidationContext();
final Collection<ValidationResult> results = validationFunction.apply(validationContext);
assertTrue(results.isEmpty());
}
@Test
void testApplyKerberosSaslWithoutCredentialsInvalid() {
runner.setProperty(KafkaClientComponent.SECURITY_PROTOCOL, SecurityProtocol.SASL_PLAINTEXT.name());
runner.setProperty(KafkaClientComponent.SASL_MECHANISM, SaslMechanism.GSSAPI.getValue());
final ValidationContext validationContext = getValidationContext();
final Collection<ValidationResult> results = validationFunction.apply(validationContext);
assertPropertyValidationResultFound(results, KafkaClientComponent.SASL_MECHANISM.getDisplayName());
}
@Test
void testApplyKerberosSaslSystemPropertyWithoutServiceNameInvalid() {
runner.setProperty(KafkaClientComponent.SECURITY_PROTOCOL, SecurityProtocol.SASL_PLAINTEXT.name());
runner.setProperty(KafkaClientComponent.SASL_MECHANISM, SaslMechanism.GSSAPI.getValue());
System.setProperty(KafkaClientCustomValidationFunction.JAVA_SECURITY_AUTH_LOGIN_CONFIG, JAAS_CONFIG_PLACEHOLDER);
final ValidationContext validationContext = getValidationContext();
final Collection<ValidationResult> results = validationFunction.apply(validationContext);
assertPropertyValidationResultFound(results, KafkaClientComponent.KERBEROS_SERVICE_NAME.getDisplayName());
}
@Test
void testApplyKerberosSaslSystemPropertyValid() {
runner.setProperty(KafkaClientComponent.SECURITY_PROTOCOL, SecurityProtocol.SASL_PLAINTEXT.name());
runner.setProperty(KafkaClientComponent.SASL_MECHANISM, SaslMechanism.GSSAPI.getValue());
runner.setProperty(KafkaClientComponent.KERBEROS_SERVICE_NAME, KafkaClientComponent.KERBEROS_SERVICE_NAME.getName());
System.setProperty(KafkaClientCustomValidationFunction.JAVA_SECURITY_AUTH_LOGIN_CONFIG, JAAS_CONFIG_PLACEHOLDER);
final ValidationContext validationContext = getValidationContext();
final Collection<ValidationResult> results = validationFunction.apply(validationContext);
assertTrue(results.isEmpty());
}
@Test
void testApplyUserServiceWithCredentialsServiceInvalid() {
runner.setProperty(KafkaClientComponent.SELF_CONTAINED_KERBEROS_USER_SERVICE, KafkaClientComponent.SELF_CONTAINED_KERBEROS_USER_SERVICE.getName());
runner.setProperty(KafkaClientComponent.KERBEROS_CREDENTIALS_SERVICE, KafkaClientComponent.KERBEROS_CREDENTIALS_SERVICE.getName());
final ValidationContext validationContext = getValidationContext();
final Collection<ValidationResult> results = validationFunction.apply(validationContext);
assertPropertyValidationResultFound(results, KafkaClientComponent.KERBEROS_CREDENTIALS_SERVICE.getDisplayName());
}
@Test
void testApplyCredentialsServiceWithPrincipalInvalid() {
runner.setProperty(KafkaClientComponent.KERBEROS_CREDENTIALS_SERVICE, KafkaClientComponent.KERBEROS_CREDENTIALS_SERVICE.getName());
runner.setProperty(KafkaClientComponent.KERBEROS_PRINCIPAL, KafkaClientComponent.KERBEROS_PRINCIPAL.getName());
final ValidationContext validationContext = getValidationContext();
final Collection<ValidationResult> results = validationFunction.apply(validationContext);
assertPropertyValidationResultFound(results, KafkaClientComponent.KERBEROS_CREDENTIALS_SERVICE.getDisplayName());
}
@Test
void testApplyPrincipalKeyTabValid() {
runner.setProperty(KafkaClientComponent.KERBEROS_PRINCIPAL, KafkaClientComponent.KERBEROS_PRINCIPAL.getName());
runner.setProperty(KafkaClientComponent.KERBEROS_KEYTAB, KafkaClientComponent.KERBEROS_KEYTAB.getName());
final ValidationContext validationContext = getValidationContext();
final Collection<ValidationResult> results = validationFunction.apply(validationContext);
assertTrue(results.isEmpty());
}
@Test
void testApplyPlainUsernameWithoutPasswordInvalid() {
runner.setProperty(KafkaClientComponent.SASL_USERNAME, KafkaClientComponent.SASL_USERNAME.getName());
runner.setProperty(KafkaClientComponent.SASL_MECHANISM, SaslMechanism.PLAIN.getValue());
final ValidationContext validationContext = getValidationContext();
final Collection<ValidationResult> results = validationFunction.apply(validationContext);
assertPropertyValidationResultFound(results, KafkaClientComponent.SASL_PASSWORD.getDisplayName());
}
@Test
void testApplyPlainPasswordWithoutUsernameInvalid() {
runner.setProperty(KafkaClientComponent.SASL_PASSWORD, KafkaClientComponent.SASL_PASSWORD.getName());
runner.setProperty(KafkaClientComponent.SASL_MECHANISM, SaslMechanism.PLAIN.getValue());
final ValidationContext validationContext = getValidationContext();
final Collection<ValidationResult> results = validationFunction.apply(validationContext);
assertPropertyValidationResultFound(results, KafkaClientComponent.SASL_USERNAME.getDisplayName());
}
@Test
void testApplySaslJaasConfigJndiLoginModuleInvalid() {
runner.setProperty(KafkaClientProperty.SASL_JAAS_CONFIG.getProperty(), JAAS_CONFIG_JNDI_LOGIN_MODULE);
final ValidationContext validationContext = getValidationContext();
final Collection<ValidationResult> results = validationFunction.apply(validationContext);
assertPropertyValidationResultFound(results, KafkaClientProperty.SASL_JAAS_CONFIG.getProperty());
}
private ValidationContext getValidationContext() {
final MockProcessContext processContext = (MockProcessContext) runner.getProcessContext();
return new MockValidationContext(processContext);
}
private void assertPropertyValidationResultFound(final Collection<ValidationResult> results, final String subject) {
final Optional<ValidationResult> validationResult = results.stream()
.filter(
result -> result.getSubject().equals(subject)
).findFirst();
assertTrue(validationResult.isPresent());
}
}

View File

@ -35,6 +35,7 @@
<module>nifi-kafka-1-0-nar</module>
<module>nifi-kafka-2-0-nar</module>
<module>nifi-kafka-2-6-nar</module>
<module>nifi-kafka-shared</module>
</modules>
<dependencyManagement>
<dependencies>
@ -53,6 +54,11 @@
<artifactId>nifi-kafka-2-6-processors</artifactId>
<version>1.19.0-SNAPSHOT</version>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-kafka-shared</artifactId>
<version>1.19.0-SNAPSHOT</version>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty</artifactId>