mirror of
https://github.com/apache/nifi.git
synced 2025-02-21 01:46:51 +00:00
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:
parent
09bc5bcb5a
commit
8e417c890a
@ -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>
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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 {
|
||||
|
@ -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 <security.protocol>://<bootstrap.servers>/topic
|
||||
*/
|
||||
static String buildTransitURI(String securityProtocol, String brokers, String topic) {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append(securityProtocol);
|
||||
builder.append("://");
|
||||
builder.append(brokers);
|
||||
builder.append("/");
|
||||
builder.append(topic);
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
|
||||
static void buildCommonKafkaProperties(final ProcessContext context, final Class<?> kafkaConfigClass, final Map<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 />
|
||||
* <LoginModuleClass> <ControlFlag> *(<OptionName>=<OptionValue>); <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;
|
||||
}
|
||||
|
||||
}
|
@ -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,29 +537,25 @@ 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 {
|
||||
try {
|
||||
final RecordReader reader = readerFactory.createRecordReader(flowFile, in, getLogger());
|
||||
final RecordSet recordSet = reader.createRecordSet();
|
||||
session.read(flowFile, in -> {
|
||||
try {
|
||||
final RecordReader reader = readerFactory.createRecordReader(flowFile, in, getLogger());
|
||||
final RecordSet recordSet = reader.createRecordSet();
|
||||
|
||||
final RecordSchema schema = writerFactory.getSchema(flowFile.getAttributes(), recordSet.getSchema());
|
||||
lease.publish(flowFile, recordSet, writerFactory, schema, messageKeyField, topic, partitioner, publishMetadataStrategy);
|
||||
} catch (final SchemaNotFoundException | MalformedRecordException e) {
|
||||
throw new ProcessException(e);
|
||||
}
|
||||
final RecordSchema schema = writerFactory.getSchema(flowFile.getAttributes(), recordSet.getSchema());
|
||||
lease.publish(flowFile, recordSet, writerFactory, schema, messageKeyField, topic, partitioner, publishMetadataStrategy);
|
||||
} 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);
|
||||
|
@ -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) {
|
||||
|
@ -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 {
|
||||
try (final InputStream in = new BufferedInputStream(rawIn)) {
|
||||
lease.publish(flowFile, in, messageKey, demarcatorBytes, topic, partition);
|
||||
}
|
||||
session.read(flowFile, rawIn -> {
|
||||
try (final InputStream in = new BufferedInputStream(rawIn)) {
|
||||
lease.publish(flowFile, in, messageKey, demarcatorBytes, topic, partition);
|
||||
}
|
||||
});
|
||||
|
||||
// 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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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 />
|
||||
* <LoginModuleClass> <ControlFlag> *(<OptionName>=<OptionValue>); <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));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
@ -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);
|
||||
|
@ -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();
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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();
|
||||
|
@ -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"));
|
||||
|
@ -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"));
|
||||
|
@ -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);
|
||||
|
@ -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
|
@ -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
|
55
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-shared/pom.xml
Normal file
55
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-shared/pom.xml
Normal 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>
|
@ -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());
|
||||
}
|
||||
String KAFKA_MAX_OFFSET = "kafka.max.offset";
|
||||
|
||||
@Test
|
||||
public void getTransactionalIdSupplierWithEmptyPrefix() {
|
||||
Supplier<String> prefix = KafkaProcessorUtils.getTransactionalIdSupplier(null);
|
||||
assertEquals(36, prefix.get().length() );
|
||||
}
|
||||
}
|
||||
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";
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
@ -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();
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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
|
||||
}
|
@ -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();
|
||||
}
|
@ -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);
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
||||
}
|
@ -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());
|
||||
}
|
||||
}
|
@ -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>
|
||||
|
Loading…
x
Reference in New Issue
Block a user