lordgamez commented on a change in pull request #940: URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r530919194
########## File path: extensions/librdkafka/ConsumeKafka.cpp ########## @@ -0,0 +1,522 @@ +/** + * 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. + */ + +#include "ConsumeKafka.h" + +#include <algorithm> +#include <limits> + +#include "core/PropertyValidation.h" +#include "utils/ProcessorConfigUtils.h" +#include "utils/gsl.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace processors { + +constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS; +constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME; + +core::Property ConsumeKafka::KafkaBrokers(core::PropertyBuilder::createProperty("Kafka Brokers") + ->withDescription("A comma-separated list of known Kafka Brokers in the format <host>:<port>.") + ->withDefaultValue("localhost:9092", core::StandardValidators::get().NON_BLANK_VALIDATOR) + ->supportsExpressionLanguage(true) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::SecurityProtocol(core::PropertyBuilder::createProperty("Security Protocol") + ->withDescription("This property is currently not supported. Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.") + ->withAllowableValues<std::string>({SECURITY_PROTOCOL_PLAINTEXT/*, SECURITY_PROTOCOL_SSL, SECURITY_PROTOCOL_SASL_PLAINTEXT, SECURITY_PROTOCOL_SASL_SSL*/ }) + ->withDefaultValue(SECURITY_PROTOCOL_PLAINTEXT) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::TopicNames(core::PropertyBuilder::createProperty("Topic Names") + ->withDescription("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma separated.") + ->supportsExpressionLanguage(true) + ->build()); + +core::Property ConsumeKafka::TopicNameFormat(core::PropertyBuilder::createProperty("Topic Name Format") + ->withDescription("Specifies whether the Topic(s) provided are a comma separated list of names or a single regular expression.") + ->withAllowableValues<std::string>({TOPIC_FORMAT_NAMES, TOPIC_FORMAT_PATTERNS}) + ->withDefaultValue(TOPIC_FORMAT_NAMES) + ->build()); + +core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions") + ->withDescription( + "Specifies whether or not NiFi should honor transactional guarantees when communicating with Kafka. If false, the Processor will use an \"isolation level\" of " + "read_uncomitted. This means that messages will be received as soon as they are written to Kafka but will be pulled, even if the producer cancels the transactions. " + "If this value is true, NiFi will not receive any messages for which the producer's transaction was canceled, but this can result in some latency since the consumer " + "must wait for the producer to finish its entire transaction instead of pulling as the messages become available.") + ->withDefaultValue<bool>(true) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID") + ->withDescription("A Group ID is used to identify consumers that are within the same consumer group. Corresponds to Kafka's 'group.id' property.") + ->supportsExpressionLanguage(true) + ->build()); + +core::Property ConsumeKafka::OffsetReset(core::PropertyBuilder::createProperty("Offset Reset") + ->withDescription("Allows you to manage the condition when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that " + "data has been deleted). Corresponds to Kafka's 'auto.offset.reset' property.") + ->withAllowableValues<std::string>({OFFSET_RESET_EARLIEST, OFFSET_RESET_LATEST, OFFSET_RESET_NONE}) + ->withDefaultValue(OFFSET_RESET_LATEST) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::KeyAttributeEncoding(core::PropertyBuilder::createProperty("Key Attribute Encoding") + ->withDescription("FlowFiles that are emitted have an attribute named 'kafka.key'. This property dictates how the value of the attribute should be encoded.") + ->withAllowableValues<std::string>({KEY_ATTR_ENCODING_UTF_8, KEY_ATTR_ENCODING_HEX}) + ->withDefaultValue(KEY_ATTR_ENCODING_UTF_8) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::MessageDemarcator(core::PropertyBuilder::createProperty("Message Demarcator") + ->withDescription("Since KafkaConsumer receives messages in batches, you have an option to output FlowFiles which contains all Kafka messages in a single batch " + "for a given topic and partition and this property allows you to provide a string (interpreted as UTF-8) to use for demarcating apart multiple Kafka messages. " + "This is an optional property and if not provided each Kafka message received will result in a single FlowFile which time it is triggered. ") + ->supportsExpressionLanguage(true) + ->build()); + +core::Property ConsumeKafka::MessageHeaderEncoding(core::PropertyBuilder::createProperty("Message Header Encoding") + ->withDescription("Any message header that is found on a Kafka message will be added to the outbound FlowFile as an attribute. This property indicates the Character Encoding " + "to use for deserializing the headers.") + ->withAllowableValues<std::string>({MSG_HEADER_ENCODING_UTF_8, MSG_HEADER_ENCODING_HEX}) + ->withDefaultValue(MSG_HEADER_ENCODING_UTF_8) + ->build()); + +core::Property ConsumeKafka::HeadersToAddAsAttributes(core::PropertyBuilder::createProperty("Headers To Add As Attributes") + ->withDescription("A Regular Expression that is matched against all message headers. Any message header whose name matches the regex will be added to the FlowFile " + "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. If two messages have a different value for the same header and that " + "header is selected by the provided regex, then those two messages must be added to different FlowFiles. As a result, users should be cautious about using a " + "regex like \".*\" if messages are expected to have header values that are unique per message, such as an identifier or timestamp, because it will prevent MiNiFi " + "from bundling the messages together efficiently.") + ->build()); + +core::Property ConsumeKafka::DuplicateHeaderHandling(core::PropertyBuilder::createProperty("Duplicate Header Handling") + ->withDescription("For headers to be added as attributes, this option specifies how to handle cases where multiple headers are present with the same key. " + "For example in case of receiving these two headers: \"Accept: text/html\" and \"Accept: application/xml\" and we want to attach the value of \"Accept\" " + "as a FlowFile attribute:\n" + " - \"Keep First\" attaches: \"Accept -> text/html\"\n" + " - \"Keep Latest\" attaches: \"Accept -> application/xml\"\n" + " - \"Comma-separated Merge\" attaches: \"Accept -> text/html, application/xml\"\n") + ->withAllowableValues<std::string>({MSG_HEADER_KEEP_FIRST, MSG_HEADER_KEEP_LATEST, MSG_HEADER_COMMA_SEPARATED_MERGE}) + ->withDefaultValue(MSG_HEADER_KEEP_LATEST) // Mirroring NiFi behaviour + ->build()); + +core::Property ConsumeKafka::MaxPollRecords(core::PropertyBuilder::createProperty("Max Poll Records") + ->withDescription("Specifies the maximum number of records Kafka should return when polling each time the processor is triggered.") + ->withDefaultValue<unsigned int>(DEFAULT_MAX_POLL_RECORDS) + ->build()); + +core::Property ConsumeKafka::MaxPollTime(core::PropertyBuilder::createProperty("Max Poll Time") + ->withDescription("Specifies the maximum amount of time the consumer can use for polling data from the brokers. " + "Polling is a blocking operation, so the upper limit of this value is specified in 4 seconds.") + ->withDefaultValue(DEFAULT_MAX_POLL_TIME, std::make_shared<core::ConsumeKafkaMaxPollTimeValidator>(std::string("ConsumeKafkaMaxPollTimeValidator"))) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::SessionTimeout(core::PropertyBuilder::createProperty("Session Timeout") + ->withDescription("Client group session and failure detection timeout. The consumer sends periodic heartbeats " + "to indicate its liveness to the broker. If no hearts are received by the broker for a group member within " + "the session timeout, the broker will remove the consumer from the group and trigger a rebalance. " + "The allowed range is configured with the broker configuration properties group.min.session.timeout.ms and group.max.session.timeout.ms.") + ->withDefaultValue<core::TimePeriodValue>("60 seconds") + ->build()); + +const core::Relationship ConsumeKafka::Success("success", "Incoming kafka messages as flowfiles. Depending on the demarcation strategy, this can be one or multiple flowfiles per message."); + +void ConsumeKafka::initialize() { + setSupportedProperties({ + KafkaBrokers, + SecurityProtocol, + TopicNames, + TopicNameFormat, + HonorTransactions, + GroupID, + OffsetReset, + KeyAttributeEncoding, + MessageDemarcator, + MessageHeaderEncoding, + HeadersToAddAsAttributes, + DuplicateHeaderHandling, + MaxPollRecords, + MaxPollTime, + SessionTimeout + }); + setSupportedRelationships({ + Success, + }); +} + +void ConsumeKafka::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /* sessionFactory */) { + // Required properties + kafka_brokers_ = utils::getRequiredPropertyOrThrow(context, KafkaBrokers.getName()); + security_protocol_ = utils::getRequiredPropertyOrThrow(context, SecurityProtocol.getName()); + topic_names_ = utils::listFromRequiredCommaSeparatedProperty(context, TopicNames.getName()); + topic_name_format_ = utils::getRequiredPropertyOrThrow(context, TopicNameFormat.getName()); + honor_transactions_ = utils::parseBooleanPropertyOrThrow(context, HonorTransactions.getName()); + group_id_ = utils::getRequiredPropertyOrThrow(context, GroupID.getName()); + offset_reset_ = utils::getRequiredPropertyOrThrow(context, OffsetReset.getName()); + key_attribute_encoding_ = utils::getRequiredPropertyOrThrow(context, KeyAttributeEncoding.getName()); + max_poll_time_milliseconds_ = utils::parseTimePropertyMSOrThrow(context, MaxPollTime.getName()); + session_timeout_milliseconds_ = utils::parseTimePropertyMSOrThrow(context, SessionTimeout.getName()); + + // Optional properties + context->getProperty(MessageDemarcator.getName(), message_demarcator_); Review comment: Shouldn't KafkaBrokers, TopicNames, GroupID and MessageDemarcator properties be extracted in the onTrigger function using flow files attributes if they support expression language? ########## File path: extensions/librdkafka/ConsumeKafka.cpp ########## @@ -0,0 +1,522 @@ +/** + * 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. + */ + +#include "ConsumeKafka.h" + +#include <algorithm> +#include <limits> + +#include "core/PropertyValidation.h" +#include "utils/ProcessorConfigUtils.h" +#include "utils/gsl.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace processors { + +constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS; +constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME; + +core::Property ConsumeKafka::KafkaBrokers(core::PropertyBuilder::createProperty("Kafka Brokers") + ->withDescription("A comma-separated list of known Kafka Brokers in the format <host>:<port>.") + ->withDefaultValue("localhost:9092", core::StandardValidators::get().NON_BLANK_VALIDATOR) + ->supportsExpressionLanguage(true) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::SecurityProtocol(core::PropertyBuilder::createProperty("Security Protocol") + ->withDescription("This property is currently not supported. Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.") + ->withAllowableValues<std::string>({SECURITY_PROTOCOL_PLAINTEXT/*, SECURITY_PROTOCOL_SSL, SECURITY_PROTOCOL_SASL_PLAINTEXT, SECURITY_PROTOCOL_SASL_SSL*/ }) + ->withDefaultValue(SECURITY_PROTOCOL_PLAINTEXT) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::TopicNames(core::PropertyBuilder::createProperty("Topic Names") + ->withDescription("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma separated.") + ->supportsExpressionLanguage(true) + ->build()); + +core::Property ConsumeKafka::TopicNameFormat(core::PropertyBuilder::createProperty("Topic Name Format") + ->withDescription("Specifies whether the Topic(s) provided are a comma separated list of names or a single regular expression.") + ->withAllowableValues<std::string>({TOPIC_FORMAT_NAMES, TOPIC_FORMAT_PATTERNS}) + ->withDefaultValue(TOPIC_FORMAT_NAMES) + ->build()); + +core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions") + ->withDescription( + "Specifies whether or not NiFi should honor transactional guarantees when communicating with Kafka. If false, the Processor will use an \"isolation level\" of " + "read_uncomitted. This means that messages will be received as soon as they are written to Kafka but will be pulled, even if the producer cancels the transactions. " + "If this value is true, NiFi will not receive any messages for which the producer's transaction was canceled, but this can result in some latency since the consumer " + "must wait for the producer to finish its entire transaction instead of pulling as the messages become available.") + ->withDefaultValue<bool>(true) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID") + ->withDescription("A Group ID is used to identify consumers that are within the same consumer group. Corresponds to Kafka's 'group.id' property.") + ->supportsExpressionLanguage(true) + ->build()); + +core::Property ConsumeKafka::OffsetReset(core::PropertyBuilder::createProperty("Offset Reset") + ->withDescription("Allows you to manage the condition when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that " + "data has been deleted). Corresponds to Kafka's 'auto.offset.reset' property.") + ->withAllowableValues<std::string>({OFFSET_RESET_EARLIEST, OFFSET_RESET_LATEST, OFFSET_RESET_NONE}) + ->withDefaultValue(OFFSET_RESET_LATEST) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::KeyAttributeEncoding(core::PropertyBuilder::createProperty("Key Attribute Encoding") + ->withDescription("FlowFiles that are emitted have an attribute named 'kafka.key'. This property dictates how the value of the attribute should be encoded.") + ->withAllowableValues<std::string>({KEY_ATTR_ENCODING_UTF_8, KEY_ATTR_ENCODING_HEX}) + ->withDefaultValue(KEY_ATTR_ENCODING_UTF_8) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::MessageDemarcator(core::PropertyBuilder::createProperty("Message Demarcator") + ->withDescription("Since KafkaConsumer receives messages in batches, you have an option to output FlowFiles which contains all Kafka messages in a single batch " + "for a given topic and partition and this property allows you to provide a string (interpreted as UTF-8) to use for demarcating apart multiple Kafka messages. " + "This is an optional property and if not provided each Kafka message received will result in a single FlowFile which time it is triggered. ") + ->supportsExpressionLanguage(true) + ->build()); + +core::Property ConsumeKafka::MessageHeaderEncoding(core::PropertyBuilder::createProperty("Message Header Encoding") + ->withDescription("Any message header that is found on a Kafka message will be added to the outbound FlowFile as an attribute. This property indicates the Character Encoding " + "to use for deserializing the headers.") + ->withAllowableValues<std::string>({MSG_HEADER_ENCODING_UTF_8, MSG_HEADER_ENCODING_HEX}) + ->withDefaultValue(MSG_HEADER_ENCODING_UTF_8) + ->build()); + +core::Property ConsumeKafka::HeadersToAddAsAttributes(core::PropertyBuilder::createProperty("Headers To Add As Attributes") + ->withDescription("A Regular Expression that is matched against all message headers. Any message header whose name matches the regex will be added to the FlowFile " + "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. If two messages have a different value for the same header and that " + "header is selected by the provided regex, then those two messages must be added to different FlowFiles. As a result, users should be cautious about using a " + "regex like \".*\" if messages are expected to have header values that are unique per message, such as an identifier or timestamp, because it will prevent MiNiFi " + "from bundling the messages together efficiently.") + ->build()); + +core::Property ConsumeKafka::DuplicateHeaderHandling(core::PropertyBuilder::createProperty("Duplicate Header Handling") + ->withDescription("For headers to be added as attributes, this option specifies how to handle cases where multiple headers are present with the same key. " + "For example in case of receiving these two headers: \"Accept: text/html\" and \"Accept: application/xml\" and we want to attach the value of \"Accept\" " + "as a FlowFile attribute:\n" + " - \"Keep First\" attaches: \"Accept -> text/html\"\n" + " - \"Keep Latest\" attaches: \"Accept -> application/xml\"\n" + " - \"Comma-separated Merge\" attaches: \"Accept -> text/html, application/xml\"\n") + ->withAllowableValues<std::string>({MSG_HEADER_KEEP_FIRST, MSG_HEADER_KEEP_LATEST, MSG_HEADER_COMMA_SEPARATED_MERGE}) + ->withDefaultValue(MSG_HEADER_KEEP_LATEST) // Mirroring NiFi behaviour + ->build()); + +core::Property ConsumeKafka::MaxPollRecords(core::PropertyBuilder::createProperty("Max Poll Records") + ->withDescription("Specifies the maximum number of records Kafka should return when polling each time the processor is triggered.") + ->withDefaultValue<unsigned int>(DEFAULT_MAX_POLL_RECORDS) + ->build()); + +core::Property ConsumeKafka::MaxPollTime(core::PropertyBuilder::createProperty("Max Poll Time") + ->withDescription("Specifies the maximum amount of time the consumer can use for polling data from the brokers. " + "Polling is a blocking operation, so the upper limit of this value is specified in 4 seconds.") + ->withDefaultValue(DEFAULT_MAX_POLL_TIME, std::make_shared<core::ConsumeKafkaMaxPollTimeValidator>(std::string("ConsumeKafkaMaxPollTimeValidator"))) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::SessionTimeout(core::PropertyBuilder::createProperty("Session Timeout") + ->withDescription("Client group session and failure detection timeout. The consumer sends periodic heartbeats " + "to indicate its liveness to the broker. If no hearts are received by the broker for a group member within " + "the session timeout, the broker will remove the consumer from the group and trigger a rebalance. " + "The allowed range is configured with the broker configuration properties group.min.session.timeout.ms and group.max.session.timeout.ms.") + ->withDefaultValue<core::TimePeriodValue>("60 seconds") + ->build()); + +const core::Relationship ConsumeKafka::Success("success", "Incoming kafka messages as flowfiles. Depending on the demarcation strategy, this can be one or multiple flowfiles per message."); + +void ConsumeKafka::initialize() { + setSupportedProperties({ + KafkaBrokers, + SecurityProtocol, + TopicNames, + TopicNameFormat, + HonorTransactions, + GroupID, + OffsetReset, + KeyAttributeEncoding, + MessageDemarcator, + MessageHeaderEncoding, + HeadersToAddAsAttributes, + DuplicateHeaderHandling, + MaxPollRecords, + MaxPollTime, + SessionTimeout + }); + setSupportedRelationships({ + Success, + }); +} + +void ConsumeKafka::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /* sessionFactory */) { + // Required properties + kafka_brokers_ = utils::getRequiredPropertyOrThrow(context, KafkaBrokers.getName()); + security_protocol_ = utils::getRequiredPropertyOrThrow(context, SecurityProtocol.getName()); + topic_names_ = utils::listFromRequiredCommaSeparatedProperty(context, TopicNames.getName()); + topic_name_format_ = utils::getRequiredPropertyOrThrow(context, TopicNameFormat.getName()); + honor_transactions_ = utils::parseBooleanPropertyOrThrow(context, HonorTransactions.getName()); + group_id_ = utils::getRequiredPropertyOrThrow(context, GroupID.getName()); + offset_reset_ = utils::getRequiredPropertyOrThrow(context, OffsetReset.getName()); + key_attribute_encoding_ = utils::getRequiredPropertyOrThrow(context, KeyAttributeEncoding.getName()); + max_poll_time_milliseconds_ = utils::parseTimePropertyMSOrThrow(context, MaxPollTime.getName()); + session_timeout_milliseconds_ = utils::parseTimePropertyMSOrThrow(context, SessionTimeout.getName()); + + // Optional properties + context->getProperty(MessageDemarcator.getName(), message_demarcator_); + context->getProperty(MessageHeaderEncoding.getName(), message_header_encoding_); + context->getProperty(DuplicateHeaderHandling.getName(), duplicate_header_handling_); + + headers_to_add_as_attributes_ = utils::listFromCommaSeparatedProperty(context, HeadersToAddAsAttributes.getName()); + max_poll_records_ = gsl::narrow<std::size_t>(utils::getOptionalUintProperty(context, MaxPollRecords.getName()).value_or(DEFAULT_MAX_POLL_RECORDS)); + + // For now security protocols are not yet supported + if (!utils::StringUtils::equalsIgnoreCase(SECURITY_PROTOCOL_PLAINTEXT, security_protocol_)) { + throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Security protocols are not supported yet."); + } + + if (!utils::StringUtils::equalsIgnoreCase(KEY_ATTR_ENCODING_UTF_8, key_attribute_encoding_) && !utils::StringUtils::equalsIgnoreCase(KEY_ATTR_ENCODING_HEX, key_attribute_encoding_)) { + throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Unsupported key attribute encoding: " + key_attribute_encoding_); + } + + if (!utils::StringUtils::equalsIgnoreCase(MSG_HEADER_ENCODING_UTF_8, message_header_encoding_) && !utils::StringUtils::equalsIgnoreCase(MSG_HEADER_ENCODING_HEX, message_header_encoding_)) { + throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Unsupported message header encoding: " + key_attribute_encoding_); + } + + configure_new_connection(context); +} + +void rebalance_cb(rd_kafka_t* rk, rd_kafka_resp_err_t trigger, rd_kafka_topic_partition_list_t* partitions, void* /*opaque*/) { + // Cooperative, incremental assignment is not supported in the current librdkafka version + std::shared_ptr<logging::Logger> logger{logging::LoggerFactory<ConsumeKafka>::getLogger()}; + logger->log_debug("Rebalance triggered."); + rd_kafka_resp_err_t assign_error = RD_KAFKA_RESP_ERR_NO_ERROR; + switch (trigger) { + case RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS: + logger->log_debug("assigned"); + utils::print_topics_list(logger, partitions); + assign_error = rd_kafka_assign(rk, partitions); + break; + + case RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS: + logger->log_debug("revoked:"); + rd_kafka_commit(rk, partitions, /* async = */ 0); // Sync commit, maybe unneccessary + utils::print_topics_list(logger, partitions); + assign_error = rd_kafka_assign(rk, NULL); + break; + + default: + logger->log_debug("failed: %s", rd_kafka_err2str(trigger)); + assign_error = rd_kafka_assign(rk, NULL); + break; + } + logger->log_debug("assign failure: %s", rd_kafka_err2str(assign_error)); +} + +void ConsumeKafka::createTopicPartitionList() { + kf_topic_partition_list_ = { rd_kafka_topic_partition_list_new(topic_names_.size()), utils::rd_kafka_topic_partition_list_deleter() }; + + // On subscriptions any topics prefixed with ^ will be regex matched + if (topic_name_format_ == "pattern") { + for (const std::string& topic : topic_names_) { + const std::string regex_format = "^" + topic; + rd_kafka_topic_partition_list_add(kf_topic_partition_list_.get(), regex_format.c_str(), RD_KAFKA_PARTITION_UA); + } + } else { + for (const std::string& topic : topic_names_) { + rd_kafka_topic_partition_list_add(kf_topic_partition_list_.get(), topic.c_str(), RD_KAFKA_PARTITION_UA); + } + } + + // Subscribe to topic set using balanced consumer groups + // Subscribing from the same process without an inbetween unsubscribe call + // Does not seem to be triggering a rebalance (maybe librdkafka bug?) + // This might happen until the cross-overship between processors and connections are settled + rd_kafka_resp_err_t subscribe_response = rd_kafka_subscribe(consumer_.get(), kf_topic_partition_list_.get()); + if (RD_KAFKA_RESP_ERR_NO_ERROR != subscribe_response) { + logger_->log_error("rd_kafka_subscribe error %d: %s", subscribe_response, rd_kafka_err2str(subscribe_response)); + } +} + +void ConsumeKafka::extend_config_from_dynamic_properties(const core::ProcessContext* context) { Review comment: Casing of the member functions seem inconsistent. IMO all member functions should be camelCased. ########## File path: extensions/librdkafka/ConsumeKafka.cpp ########## @@ -0,0 +1,522 @@ +/** + * 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. + */ + +#include "ConsumeKafka.h" + +#include <algorithm> +#include <limits> + +#include "core/PropertyValidation.h" +#include "utils/ProcessorConfigUtils.h" +#include "utils/gsl.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace processors { + +constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS; +constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME; + +core::Property ConsumeKafka::KafkaBrokers(core::PropertyBuilder::createProperty("Kafka Brokers") + ->withDescription("A comma-separated list of known Kafka Brokers in the format <host>:<port>.") + ->withDefaultValue("localhost:9092", core::StandardValidators::get().NON_BLANK_VALIDATOR) + ->supportsExpressionLanguage(true) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::SecurityProtocol(core::PropertyBuilder::createProperty("Security Protocol") + ->withDescription("This property is currently not supported. Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.") + ->withAllowableValues<std::string>({SECURITY_PROTOCOL_PLAINTEXT/*, SECURITY_PROTOCOL_SSL, SECURITY_PROTOCOL_SASL_PLAINTEXT, SECURITY_PROTOCOL_SASL_SSL*/ }) + ->withDefaultValue(SECURITY_PROTOCOL_PLAINTEXT) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::TopicNames(core::PropertyBuilder::createProperty("Topic Names") + ->withDescription("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma separated.") + ->supportsExpressionLanguage(true) + ->build()); + +core::Property ConsumeKafka::TopicNameFormat(core::PropertyBuilder::createProperty("Topic Name Format") + ->withDescription("Specifies whether the Topic(s) provided are a comma separated list of names or a single regular expression.") + ->withAllowableValues<std::string>({TOPIC_FORMAT_NAMES, TOPIC_FORMAT_PATTERNS}) + ->withDefaultValue(TOPIC_FORMAT_NAMES) + ->build()); + +core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions") + ->withDescription( + "Specifies whether or not NiFi should honor transactional guarantees when communicating with Kafka. If false, the Processor will use an \"isolation level\" of " + "read_uncomitted. This means that messages will be received as soon as they are written to Kafka but will be pulled, even if the producer cancels the transactions. " + "If this value is true, NiFi will not receive any messages for which the producer's transaction was canceled, but this can result in some latency since the consumer " + "must wait for the producer to finish its entire transaction instead of pulling as the messages become available.") + ->withDefaultValue<bool>(true) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID") + ->withDescription("A Group ID is used to identify consumers that are within the same consumer group. Corresponds to Kafka's 'group.id' property.") + ->supportsExpressionLanguage(true) + ->build()); + +core::Property ConsumeKafka::OffsetReset(core::PropertyBuilder::createProperty("Offset Reset") + ->withDescription("Allows you to manage the condition when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that " + "data has been deleted). Corresponds to Kafka's 'auto.offset.reset' property.") + ->withAllowableValues<std::string>({OFFSET_RESET_EARLIEST, OFFSET_RESET_LATEST, OFFSET_RESET_NONE}) + ->withDefaultValue(OFFSET_RESET_LATEST) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::KeyAttributeEncoding(core::PropertyBuilder::createProperty("Key Attribute Encoding") + ->withDescription("FlowFiles that are emitted have an attribute named 'kafka.key'. This property dictates how the value of the attribute should be encoded.") + ->withAllowableValues<std::string>({KEY_ATTR_ENCODING_UTF_8, KEY_ATTR_ENCODING_HEX}) + ->withDefaultValue(KEY_ATTR_ENCODING_UTF_8) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::MessageDemarcator(core::PropertyBuilder::createProperty("Message Demarcator") + ->withDescription("Since KafkaConsumer receives messages in batches, you have an option to output FlowFiles which contains all Kafka messages in a single batch " + "for a given topic and partition and this property allows you to provide a string (interpreted as UTF-8) to use for demarcating apart multiple Kafka messages. " + "This is an optional property and if not provided each Kafka message received will result in a single FlowFile which time it is triggered. ") + ->supportsExpressionLanguage(true) + ->build()); + +core::Property ConsumeKafka::MessageHeaderEncoding(core::PropertyBuilder::createProperty("Message Header Encoding") + ->withDescription("Any message header that is found on a Kafka message will be added to the outbound FlowFile as an attribute. This property indicates the Character Encoding " + "to use for deserializing the headers.") + ->withAllowableValues<std::string>({MSG_HEADER_ENCODING_UTF_8, MSG_HEADER_ENCODING_HEX}) + ->withDefaultValue(MSG_HEADER_ENCODING_UTF_8) + ->build()); + +core::Property ConsumeKafka::HeadersToAddAsAttributes(core::PropertyBuilder::createProperty("Headers To Add As Attributes") + ->withDescription("A Regular Expression that is matched against all message headers. Any message header whose name matches the regex will be added to the FlowFile " + "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. If two messages have a different value for the same header and that " + "header is selected by the provided regex, then those two messages must be added to different FlowFiles. As a result, users should be cautious about using a " + "regex like \".*\" if messages are expected to have header values that are unique per message, such as an identifier or timestamp, because it will prevent MiNiFi " + "from bundling the messages together efficiently.") + ->build()); + +core::Property ConsumeKafka::DuplicateHeaderHandling(core::PropertyBuilder::createProperty("Duplicate Header Handling") + ->withDescription("For headers to be added as attributes, this option specifies how to handle cases where multiple headers are present with the same key. " + "For example in case of receiving these two headers: \"Accept: text/html\" and \"Accept: application/xml\" and we want to attach the value of \"Accept\" " + "as a FlowFile attribute:\n" + " - \"Keep First\" attaches: \"Accept -> text/html\"\n" + " - \"Keep Latest\" attaches: \"Accept -> application/xml\"\n" + " - \"Comma-separated Merge\" attaches: \"Accept -> text/html, application/xml\"\n") + ->withAllowableValues<std::string>({MSG_HEADER_KEEP_FIRST, MSG_HEADER_KEEP_LATEST, MSG_HEADER_COMMA_SEPARATED_MERGE}) + ->withDefaultValue(MSG_HEADER_KEEP_LATEST) // Mirroring NiFi behaviour + ->build()); + +core::Property ConsumeKafka::MaxPollRecords(core::PropertyBuilder::createProperty("Max Poll Records") + ->withDescription("Specifies the maximum number of records Kafka should return when polling each time the processor is triggered.") + ->withDefaultValue<unsigned int>(DEFAULT_MAX_POLL_RECORDS) + ->build()); + +core::Property ConsumeKafka::MaxPollTime(core::PropertyBuilder::createProperty("Max Poll Time") + ->withDescription("Specifies the maximum amount of time the consumer can use for polling data from the brokers. " + "Polling is a blocking operation, so the upper limit of this value is specified in 4 seconds.") + ->withDefaultValue(DEFAULT_MAX_POLL_TIME, std::make_shared<core::ConsumeKafkaMaxPollTimeValidator>(std::string("ConsumeKafkaMaxPollTimeValidator"))) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::SessionTimeout(core::PropertyBuilder::createProperty("Session Timeout") + ->withDescription("Client group session and failure detection timeout. The consumer sends periodic heartbeats " + "to indicate its liveness to the broker. If no hearts are received by the broker for a group member within " + "the session timeout, the broker will remove the consumer from the group and trigger a rebalance. " + "The allowed range is configured with the broker configuration properties group.min.session.timeout.ms and group.max.session.timeout.ms.") + ->withDefaultValue<core::TimePeriodValue>("60 seconds") + ->build()); + +const core::Relationship ConsumeKafka::Success("success", "Incoming kafka messages as flowfiles. Depending on the demarcation strategy, this can be one or multiple flowfiles per message."); + +void ConsumeKafka::initialize() { + setSupportedProperties({ + KafkaBrokers, + SecurityProtocol, + TopicNames, + TopicNameFormat, + HonorTransactions, + GroupID, + OffsetReset, + KeyAttributeEncoding, + MessageDemarcator, + MessageHeaderEncoding, + HeadersToAddAsAttributes, + DuplicateHeaderHandling, + MaxPollRecords, + MaxPollTime, + SessionTimeout + }); + setSupportedRelationships({ + Success, + }); +} + +void ConsumeKafka::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /* sessionFactory */) { + // Required properties + kafka_brokers_ = utils::getRequiredPropertyOrThrow(context, KafkaBrokers.getName()); + security_protocol_ = utils::getRequiredPropertyOrThrow(context, SecurityProtocol.getName()); + topic_names_ = utils::listFromRequiredCommaSeparatedProperty(context, TopicNames.getName()); + topic_name_format_ = utils::getRequiredPropertyOrThrow(context, TopicNameFormat.getName()); + honor_transactions_ = utils::parseBooleanPropertyOrThrow(context, HonorTransactions.getName()); + group_id_ = utils::getRequiredPropertyOrThrow(context, GroupID.getName()); + offset_reset_ = utils::getRequiredPropertyOrThrow(context, OffsetReset.getName()); + key_attribute_encoding_ = utils::getRequiredPropertyOrThrow(context, KeyAttributeEncoding.getName()); + max_poll_time_milliseconds_ = utils::parseTimePropertyMSOrThrow(context, MaxPollTime.getName()); + session_timeout_milliseconds_ = utils::parseTimePropertyMSOrThrow(context, SessionTimeout.getName()); + + // Optional properties + context->getProperty(MessageDemarcator.getName(), message_demarcator_); + context->getProperty(MessageHeaderEncoding.getName(), message_header_encoding_); + context->getProperty(DuplicateHeaderHandling.getName(), duplicate_header_handling_); + + headers_to_add_as_attributes_ = utils::listFromCommaSeparatedProperty(context, HeadersToAddAsAttributes.getName()); + max_poll_records_ = gsl::narrow<std::size_t>(utils::getOptionalUintProperty(context, MaxPollRecords.getName()).value_or(DEFAULT_MAX_POLL_RECORDS)); + + // For now security protocols are not yet supported + if (!utils::StringUtils::equalsIgnoreCase(SECURITY_PROTOCOL_PLAINTEXT, security_protocol_)) { + throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Security protocols are not supported yet."); + } + + if (!utils::StringUtils::equalsIgnoreCase(KEY_ATTR_ENCODING_UTF_8, key_attribute_encoding_) && !utils::StringUtils::equalsIgnoreCase(KEY_ATTR_ENCODING_HEX, key_attribute_encoding_)) { + throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Unsupported key attribute encoding: " + key_attribute_encoding_); + } + + if (!utils::StringUtils::equalsIgnoreCase(MSG_HEADER_ENCODING_UTF_8, message_header_encoding_) && !utils::StringUtils::equalsIgnoreCase(MSG_HEADER_ENCODING_HEX, message_header_encoding_)) { + throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Unsupported message header encoding: " + key_attribute_encoding_); + } + + configure_new_connection(context); +} + +void rebalance_cb(rd_kafka_t* rk, rd_kafka_resp_err_t trigger, rd_kafka_topic_partition_list_t* partitions, void* /*opaque*/) { + // Cooperative, incremental assignment is not supported in the current librdkafka version + std::shared_ptr<logging::Logger> logger{logging::LoggerFactory<ConsumeKafka>::getLogger()}; + logger->log_debug("Rebalance triggered."); + rd_kafka_resp_err_t assign_error = RD_KAFKA_RESP_ERR_NO_ERROR; + switch (trigger) { + case RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS: + logger->log_debug("assigned"); + utils::print_topics_list(logger, partitions); + assign_error = rd_kafka_assign(rk, partitions); + break; + + case RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS: + logger->log_debug("revoked:"); + rd_kafka_commit(rk, partitions, /* async = */ 0); // Sync commit, maybe unneccessary + utils::print_topics_list(logger, partitions); + assign_error = rd_kafka_assign(rk, NULL); + break; + + default: + logger->log_debug("failed: %s", rd_kafka_err2str(trigger)); + assign_error = rd_kafka_assign(rk, NULL); + break; + } + logger->log_debug("assign failure: %s", rd_kafka_err2str(assign_error)); +} + +void ConsumeKafka::createTopicPartitionList() { + kf_topic_partition_list_ = { rd_kafka_topic_partition_list_new(topic_names_.size()), utils::rd_kafka_topic_partition_list_deleter() }; + + // On subscriptions any topics prefixed with ^ will be regex matched + if (topic_name_format_ == "pattern") { Review comment: "pattern" does not seem to be an allowed value for this property. Shouldn't TOPIC_FORMAT_PATTERNS be used here? ########## File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp ########## @@ -0,0 +1,595 @@ + +/** + * 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. + */ + +#define CATCH_CONFIG_MAIN + +#include <algorithm> +#include <memory> +#include <string> +#include <set> + +// #include "TestBase.h" Review comment: This can be removed ########## File path: extensions/librdkafka/ConsumeKafka.cpp ########## @@ -0,0 +1,522 @@ +/** + * 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. + */ + +#include "ConsumeKafka.h" + +#include <algorithm> +#include <limits> + +#include "core/PropertyValidation.h" +#include "utils/ProcessorConfigUtils.h" +#include "utils/gsl.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace processors { + +constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS; +constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME; + +core::Property ConsumeKafka::KafkaBrokers(core::PropertyBuilder::createProperty("Kafka Brokers") + ->withDescription("A comma-separated list of known Kafka Brokers in the format <host>:<port>.") + ->withDefaultValue("localhost:9092", core::StandardValidators::get().NON_BLANK_VALIDATOR) + ->supportsExpressionLanguage(true) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::SecurityProtocol(core::PropertyBuilder::createProperty("Security Protocol") + ->withDescription("This property is currently not supported. Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.") + ->withAllowableValues<std::string>({SECURITY_PROTOCOL_PLAINTEXT/*, SECURITY_PROTOCOL_SSL, SECURITY_PROTOCOL_SASL_PLAINTEXT, SECURITY_PROTOCOL_SASL_SSL*/ }) + ->withDefaultValue(SECURITY_PROTOCOL_PLAINTEXT) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::TopicNames(core::PropertyBuilder::createProperty("Topic Names") + ->withDescription("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma separated.") + ->supportsExpressionLanguage(true) + ->build()); + +core::Property ConsumeKafka::TopicNameFormat(core::PropertyBuilder::createProperty("Topic Name Format") + ->withDescription("Specifies whether the Topic(s) provided are a comma separated list of names or a single regular expression.") + ->withAllowableValues<std::string>({TOPIC_FORMAT_NAMES, TOPIC_FORMAT_PATTERNS}) + ->withDefaultValue(TOPIC_FORMAT_NAMES) + ->build()); + +core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions") + ->withDescription( + "Specifies whether or not NiFi should honor transactional guarantees when communicating with Kafka. If false, the Processor will use an \"isolation level\" of " + "read_uncomitted. This means that messages will be received as soon as they are written to Kafka but will be pulled, even if the producer cancels the transactions. " + "If this value is true, NiFi will not receive any messages for which the producer's transaction was canceled, but this can result in some latency since the consumer " + "must wait for the producer to finish its entire transaction instead of pulling as the messages become available.") Review comment: NiFi should be replaced with Minifi ########## File path: libminifi/include/utils/ProcessorConfigUtils.h ########## @@ -0,0 +1,76 @@ +/** + * 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. + */ + +#pragma once + +#include <vector> +#include <string> + +#include "utils/StringUtils.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace utils { + +std::string getRequiredPropertyOrThrow(const core::ProcessContext* context, const std::string& property_name) { Review comment: It's great to finally have these utils :+1: ########## File path: extensions/librdkafka/ConsumeKafka.cpp ########## @@ -0,0 +1,522 @@ +/** + * 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. + */ + +#include "ConsumeKafka.h" + +#include <algorithm> +#include <limits> + +#include "core/PropertyValidation.h" +#include "utils/ProcessorConfigUtils.h" +#include "utils/gsl.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace processors { + +constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS; +constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME; + +core::Property ConsumeKafka::KafkaBrokers(core::PropertyBuilder::createProperty("Kafka Brokers") + ->withDescription("A comma-separated list of known Kafka Brokers in the format <host>:<port>.") + ->withDefaultValue("localhost:9092", core::StandardValidators::get().NON_BLANK_VALIDATOR) + ->supportsExpressionLanguage(true) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::SecurityProtocol(core::PropertyBuilder::createProperty("Security Protocol") + ->withDescription("This property is currently not supported. Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.") + ->withAllowableValues<std::string>({SECURITY_PROTOCOL_PLAINTEXT/*, SECURITY_PROTOCOL_SSL, SECURITY_PROTOCOL_SASL_PLAINTEXT, SECURITY_PROTOCOL_SASL_SSL*/ }) + ->withDefaultValue(SECURITY_PROTOCOL_PLAINTEXT) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::TopicNames(core::PropertyBuilder::createProperty("Topic Names") + ->withDescription("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma separated.") Review comment: "Multiple topics can be supplied separated by commas." may sound more natural. ########## File path: extensions/librdkafka/tests/CMakeLists.txt ########## @@ -29,8 +29,11 @@ FOREACH(testfile ${KAFKA_TESTS}) createTests("${testfilename}") MATH(EXPR KAFKA_TEST_COUNT "${KAFKA_TEST_COUNT}+1") # The line below handles integration test - add_test(NAME "${testfilename}" COMMAND "${testfilename}" "${TEST_RESOURCES}/TestKafkaOnSchedule.yml" "${TEST_RESOURCES}/") + target_include_directories(${testfilename} BEFORE PRIVATE "../../standard-processors/processors") Review comment: Tab was added here instead of spaces. ########## File path: extensions/librdkafka/ConsumeKafka.h ########## @@ -0,0 +1,197 @@ +/** + * 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. + */ + +#pragma once + +#include <string> +#include <utility> +#include <vector> +#include <memory> + +#include "core/Processor.h" +#include "core/logging/LoggerConfiguration.h" +#include "rdkafka.h" +#include "rdkafka_utils.h" +#include "KafkaConnection.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace core { + +// The upper limit for Max Poll Time is 4 seconds. This is because Watchdog would potentially start +// reporting issues with the processor health otherwise +class ConsumeKafkaMaxPollTimeValidator : public TimePeriodValidator { + public: + ConsumeKafkaMaxPollTimeValidator(const std::string &name) // NOLINT + : TimePeriodValidator(name) { + } + ~ConsumeKafkaMaxPollTimeValidator() override = default; + + ValidationResult validate(const std::string& subject, const std::string& input) const override { + uint64_t value; + TimeUnit timeUnit; + uint64_t value_as_ms; + return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input).isValid( + core::TimePeriodValue::StringToTime(input, value, timeUnit) && + org::apache::nifi::minifi::core::Property::ConvertTimeUnitToMS(value, timeUnit, value_as_ms) && + 0 < value_as_ms && value_as_ms <= 4000).build(); + } +}; + +} // namespace core +namespace processors { + +class ConsumeKafka : public core::Processor { + public: + static constexpr char const* ProcessorName = "ConsumeKafka"; + + // Supported Properties + static core::Property KafkaBrokers; + static core::Property SecurityProtocol; + static core::Property TopicNames; + static core::Property TopicNameFormat; + static core::Property HonorTransactions; + static core::Property GroupID; + static core::Property OffsetReset; + static core::Property KeyAttributeEncoding; + static core::Property MessageDemarcator; + static core::Property MessageHeaderEncoding; + static core::Property HeadersToAddAsAttributes; + static core::Property DuplicateHeaderHandling; + static core::Property MaxPollRecords; + static core::Property MaxPollTime; + static core::Property SessionTimeout; + + // Supported Relationships + static const core::Relationship Success; + + // Security Protocol allowable values + static constexpr char const* SECURITY_PROTOCOL_PLAINTEXT = "PLAINTEXT"; + static constexpr char const* SECURITY_PROTOCOL_SSL = "SSL"; + static constexpr char const* SECURITY_PROTOCOL_SASL_PLAINTEXT = "SASL_PLAINTEXT"; + static constexpr char const* SECURITY_PROTOCOL_SASL_SSL = "SASL_SSL"; + + // Topic Name Format allowable values + static constexpr char const* TOPIC_FORMAT_NAMES = "Names"; + static constexpr char const* TOPIC_FORMAT_PATTERNS = "Patterns"; + + // Offset Reset allowable values + static constexpr char const* OFFSET_RESET_EARLIEST = "earliest"; + static constexpr char const* OFFSET_RESET_LATEST = "latest"; + static constexpr char const* OFFSET_RESET_NONE = "none"; + + // Key Attribute Encoding allowable values + static constexpr char const* KEY_ATTR_ENCODING_UTF_8 = "UTF-8"; + static constexpr char const* KEY_ATTR_ENCODING_HEX = "Hex"; + + // Message Header Encoding allowable values + static constexpr char const* MSG_HEADER_ENCODING_UTF_8 = "UTF-8"; + static constexpr char const* MSG_HEADER_ENCODING_HEX = "Hex"; + + // Duplicate Header Handling allowable values + static constexpr char const* MSG_HEADER_KEEP_FIRST = "Keep First"; + static constexpr char const* MSG_HEADER_KEEP_LATEST = "Keep Latest"; + static constexpr char const* MSG_HEADER_COMMA_SEPARATED_MERGE = "Comma-separated Merge"; + + // Flowfile attributes written + static constexpr char const* KAFKA_COUNT_ATTR = "kafka.count"; // Always 1 until we start supporting merging from batches + static constexpr char const* KAFKA_MESSAGE_KEY_ATTR = "kafka.key"; + static constexpr char const* KAFKA_OFFSET_ATTR = "kafka.offset"; + static constexpr char const* KAFKA_PARTITION_ATTR = "kafka.partition"; + static constexpr char const* KAFKA_TOPIC_ATTR = "kafka.topic"; + + static constexpr const std::size_t DEFAULT_MAX_POLL_RECORDS{ 10000 }; + static constexpr char const* DEFAULT_MAX_POLL_TIME = "4 seconds"; + static constexpr const std::size_t METADATA_COMMUNICATIONS_TIMEOUT_MS{ 60000 }; + + explicit ConsumeKafka(std::string name, utils::Identifier uuid = utils::Identifier()) : + Processor(name, uuid), + logger_(logging::LoggerFactory<ConsumeKafka>::getLogger()) {} + + virtual ~ConsumeKafka() = default; + + public: + bool supportsDynamicProperties() override { + return true; + } + /** + * Function that's executed when the processor is scheduled. + * @param context process context. + * @param sessionFactory process session factory that is used when creating + * ProcessSession objects. + */ + void onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /* sessionFactory */) override; + /** + * Execution trigger for the RetryFlowFile Processor + * @param context processor context + * @param session processor session reference. + */ + void onTrigger(core::ProcessContext* context, core::ProcessSession* session) override; + + // Initialize, overwrite by NiFi RetryFlowFile + void initialize() override; + + private: + void createTopicPartitionList(); + void extend_config_from_dynamic_properties(const core::ProcessContext* context); + void configure_new_connection(const core::ProcessContext* context); + std::string extract_message(const rd_kafka_message_t* rkmessage); + std::vector<std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>> poll_kafka_messages(); + utils::KafkaEncoding key_attr_encoding_attr_to_enum(); + utils::KafkaEncoding message_header_encoding_attr_to_enum(); + std::string resolve_duplicate_headers(const std::vector<std::string>& matching_headers); + std::vector<std::string> get_matching_headers(const rd_kafka_message_t* message, const std::string& header_name); + std::vector<std::pair<std::string, std::string>> get_flowfile_attributes_from_message_header(const rd_kafka_message_t* message); + std::vector<std::shared_ptr<FlowFileRecord>> transform_messages_into_flowfiles( + const std::vector<std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>>& messages, core::ProcessSession* session); Review comment: Could these members be const? ########## File path: extensions/librdkafka/rdkafka_utils.h ########## @@ -0,0 +1,104 @@ +/** + * 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. + */ + +#pragma once + +#include <algorithm> +#include <chrono> +#include <memory> +#include <string> +#include <thread> +#include <utility> +#include <vector> + +#include "core/logging/LoggerConfiguration.h" +#include "utils/OptionalUtils.h" +#include "rdkafka.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace utils { + +enum class KafkaEncoding { + UTF8, + HEX +}; + +struct rd_kafka_conf_deleter { + void operator()(rd_kafka_conf_t* ptr) const noexcept { rd_kafka_conf_destroy(ptr); } +}; + +struct rd_kafka_producer_deleter { + void operator()(rd_kafka_t* ptr) const noexcept { + rd_kafka_resp_err_t flush_ret = rd_kafka_flush(ptr, 10000 /* ms */); // Matching the wait time of KafkaConnection.cpp + // If concerned, we could log potential errors here: + // if (RD_KAFKA_RESP_ERR__TIMED_OUT == flush_ret) { + // std::cerr << "Deleting producer failed: time-out while trying to flush" << std::endl; + // } Review comment: IMO we should not leave commented code in production. It may be logged on debug or trace level. ########## File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp ########## @@ -0,0 +1,595 @@ + +/** + * 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. + */ + +#define CATCH_CONFIG_MAIN + +#include <algorithm> +#include <memory> +#include <string> +#include <set> + +// #include "TestBase.h" +#include "../../../libminifi/test/TestBase.h" + +#include "../ConsumeKafka.h" +#include "../rdkafka_utils.h" +#include "../../standard-processors/processors/ExtractText.h" +#include "utils/file/FileUtils.h" +#include "utils/OptionalUtils.h" +#include "utils/RegexUtils.h" +#include "utils/StringUtils.h" +#include "utils/TestUtils.h" + +#include "utils/IntegrationTestUtils.h" + +namespace { +using org::apache::nifi::minifi::utils::optional; + +class KafkaTestProducer { + public: + enum class PublishEvent { + PUBLISH, + TRANSACTION_START, + TRANSACTION_COMMIT, + CANCEL + }; + KafkaTestProducer(const std::string& kafka_brokers, const std::string& topic, const bool transactional) : + logger_(logging::LoggerFactory<KafkaTestProducer>::getLogger()) { + using utils::setKafkaConfigurationField; + + std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf = { rd_kafka_conf_new(), utils::rd_kafka_conf_deleter() }; + + setKafkaConfigurationField(conf.get(), "bootstrap.servers", kafka_brokers); + // setKafkaConfigurationField(conf.get(), "client.id", PRODUCER_CLIENT_NAME); Review comment: Is this needed? ########## File path: libminifi/test/TestBase.cpp ########## @@ -247,45 +221,65 @@ void TestPlan::reset(bool reschedule) { } } -bool TestPlan::runNextProcessor(std::function<void(const std::shared_ptr<core::ProcessContext>, const std::shared_ptr<core::ProcessSession>)> verify) { - if (!finalized) { - finalize(); +std::vector<std::shared_ptr<core::Processor>>::iterator TestPlan::getProcessorItByUuid(const std::string& uuid) { + const auto processor_node_matches_processor = [&uuid] (const std::shared_ptr<core::Processor>& processor) { + return processor->getUUIDStr() == uuid; + }; + auto processor_found_at = std::find_if(processor_queue_.begin(), processor_queue_.end(), processor_node_matches_processor); + if (processor_found_at == processor_queue_.end()) { + throw std::runtime_error("Processor not found in test plan."); } - logger_->log_info("Running next processor %d, processor_queue_.size %d, processor_contexts_.size %d", location, processor_queue_.size(), processor_contexts_.size()); - std::lock_guard<std::recursive_mutex> guard(mutex); - location++; - std::shared_ptr<core::Processor> processor = processor_queue_.at(location); - std::shared_ptr<core::ProcessContext> context = processor_contexts_.at(location); - std::shared_ptr<core::ProcessSessionFactory> factory = std::make_shared<core::ProcessSessionFactory>(context); - factories_.push_back(factory); + return processor_found_at; +} + +std::shared_ptr<core::ProcessContext> TestPlan::getProcessContextForProcessor(const std::shared_ptr<core::Processor>& processor) { + const auto contextMatchesProcessor = [&processor] (const std::shared_ptr<core::ProcessContext>& context) { + return context->getProcessorNode()->getUUIDStr() == processor->getUUIDStr(); + }; + const auto context_found_at = std::find_if(processor_contexts_.begin(), processor_contexts_.end(), contextMatchesProcessor); + if (context_found_at == processor_contexts_.end()) { + throw std::runtime_error("Context not found in test plan."); + } + return *context_found_at; +} + +void TestPlan::schedule_processors() { + for(std::size_t target_location = 0; target_location < processor_queue_.size(); ++target_location) { + std::shared_ptr<core::Processor> processor = processor_queue_.at(target_location); + std::shared_ptr<core::ProcessContext> context = processor_contexts_.at(target_location); + schedule_processor(processor, context); + } +} + +void TestPlan::schedule_processor(const std::shared_ptr<core::Processor>& processor) { + schedule_processor(processor, getProcessContextForProcessor(processor)); +} + +void TestPlan::schedule_processor(const std::shared_ptr<core::Processor>& processor, const std::shared_ptr<core::ProcessContext>& context) { Review comment: Some inconsistency in casing with the other camelCase member funtions. ########## File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp ########## @@ -0,0 +1,595 @@ + +/** + * 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. + */ + +#define CATCH_CONFIG_MAIN + +#include <algorithm> +#include <memory> +#include <string> +#include <set> + +// #include "TestBase.h" +#include "../../../libminifi/test/TestBase.h" + +#include "../ConsumeKafka.h" +#include "../rdkafka_utils.h" +#include "../../standard-processors/processors/ExtractText.h" +#include "utils/file/FileUtils.h" +#include "utils/OptionalUtils.h" +#include "utils/RegexUtils.h" +#include "utils/StringUtils.h" +#include "utils/TestUtils.h" + +#include "utils/IntegrationTestUtils.h" + +namespace { +using org::apache::nifi::minifi::utils::optional; + +class KafkaTestProducer { + public: + enum class PublishEvent { + PUBLISH, + TRANSACTION_START, + TRANSACTION_COMMIT, + CANCEL + }; + KafkaTestProducer(const std::string& kafka_brokers, const std::string& topic, const bool transactional) : + logger_(logging::LoggerFactory<KafkaTestProducer>::getLogger()) { + using utils::setKafkaConfigurationField; + + std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf = { rd_kafka_conf_new(), utils::rd_kafka_conf_deleter() }; + + setKafkaConfigurationField(conf.get(), "bootstrap.servers", kafka_brokers); + // setKafkaConfigurationField(conf.get(), "client.id", PRODUCER_CLIENT_NAME); + setKafkaConfigurationField(conf.get(), "compression.codec", "snappy"); + setKafkaConfigurationField(conf.get(), "batch.num.messages", "1"); + + if (transactional) { + setKafkaConfigurationField(conf.get(), "transactional.id", "ConsumeKafkaTest_transaction_id"); + } + + static std::array<char, 512U> errstr{}; + producer_ = { rd_kafka_new(RD_KAFKA_PRODUCER, conf.release(), errstr.data(), errstr.size()), utils::rd_kafka_producer_deleter() }; + if (producer_ == nullptr) { + auto error_msg = utils::StringUtils::join_pack("Failed to create Kafka producer %s", errstr.data()); + throw std::runtime_error(error_msg); + } + + // The last argument is a config here, but it is already owned by the consumer. I assume that this would mean an override on the original config if used + topic_ = { rd_kafka_topic_new(producer_.get(), topic.c_str(), nullptr), utils::rd_kafka_topic_deleter() }; + + if (transactional) { + rd_kafka_init_transactions(producer_.get(), TRANSACTIONS_TIMEOUT_MS.count()); + } + } + + // Uses all the headers for every published message + void publish_messages_to_topic( + const std::vector<std::string>& messages_on_topic, const std::string& message_key, std::vector<PublishEvent> events, + const std::vector<std::pair<std::string, std::string>>& message_headers, const optional<std::string>& message_header_encoding) { + auto next_message = messages_on_topic.cbegin(); + for (const PublishEvent event : events) { + switch (event) { + case PublishEvent::PUBLISH: + REQUIRE(messages_on_topic.cend() != next_message); + publish_message(*next_message, message_key, message_headers, message_header_encoding); + std::advance(next_message, 1); + break; + case PublishEvent::TRANSACTION_START: + logger_->log_debug("Starting new transaction..."); + rd_kafka_begin_transaction(producer_.get()); + break; + case PublishEvent::TRANSACTION_COMMIT: + logger_->log_debug("Committing transaction..."); + rd_kafka_commit_transaction(producer_.get(), TRANSACTIONS_TIMEOUT_MS.count()); + break; + case PublishEvent::CANCEL: + logger_->log_debug("Cancelling transaction..."); + rd_kafka_abort_transaction(producer_.get(), TRANSACTIONS_TIMEOUT_MS.count()); + } + } + } + + private: + void publish_message( + const std::string& message, const std::string& message_key, const std::vector<std::pair<std::string, std::string>>& message_headers, const optional<std::string>& message_header_encoding) { + logger_->log_debug("Producing: %s", message.c_str()); + std::unique_ptr<rd_kafka_headers_t, utils::rd_kafka_headers_deleter> headers(rd_kafka_headers_new(message_headers.size()), utils::rd_kafka_headers_deleter()); + if (!headers) { + throw std::runtime_error("Generating message headers failed."); + } + for (const std::pair<std::string, std::string>& message_header : message_headers) { + rd_kafka_header_add(headers.get(), + const_cast<char*>(message_header.first.c_str()), message_header.first.size(), + const_cast<char*>(message_header.second.c_str()), message_header.second.size()); + } + + if (RD_KAFKA_RESP_ERR_NO_ERROR != rd_kafka_producev( + producer_.get(), + RD_KAFKA_V_RKT(topic_.get()), + RD_KAFKA_V_PARTITION(RD_KAFKA_PARTITION_UA), + RD_KAFKA_V_MSGFLAGS(RD_KAFKA_MSG_F_COPY), + RD_KAFKA_V_VALUE(const_cast<char*>(&message[0]), message.size()), + RD_KAFKA_V_HEADERS(headers.release()), + RD_KAFKA_V_KEY(message_key.c_str(), message_key.size()), + RD_KAFKA_V_END)) { + logger_->log_error("Producer failure: %d: %s", rd_kafka_last_error(), rd_kafka_err2str(rd_kafka_last_error())); + } + } + + static const std::chrono::milliseconds TRANSACTIONS_TIMEOUT_MS; + + std::unique_ptr<rd_kafka_t, utils::rd_kafka_producer_deleter> producer_; + std::unique_ptr<rd_kafka_topic_t, utils::rd_kafka_topic_deleter> topic_; + + std::shared_ptr<logging::Logger> logger_; +}; + +const std::chrono::milliseconds KafkaTestProducer::TRANSACTIONS_TIMEOUT_MS{ 2000 }; + +class ConsumeKafkaTest { + public: + using Processor = org::apache::nifi::minifi::core::Processor; + using ConsumeKafka = org::apache::nifi::minifi::processors::ConsumeKafka; + using ExtractText = org::apache::nifi::minifi::processors::ExtractText; + + const KafkaTestProducer::PublishEvent PUBLISH = KafkaTestProducer::PublishEvent::PUBLISH; + const KafkaTestProducer::PublishEvent TRANSACTION_START = KafkaTestProducer::PublishEvent::TRANSACTION_START; + const KafkaTestProducer::PublishEvent TRANSACTION_COMMIT = KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT; + const KafkaTestProducer::PublishEvent CANCEL = KafkaTestProducer::PublishEvent::CANCEL; + + const std::vector<KafkaTestProducer::PublishEvent> NON_TRANSACTIONAL_MESSAGES { PUBLISH, PUBLISH }; + const std::vector<KafkaTestProducer::PublishEvent> SINGLE_COMMITTED_TRANSACTION { TRANSACTION_START, PUBLISH, PUBLISH, TRANSACTION_COMMIT }; + const std::vector<KafkaTestProducer::PublishEvent> TWO_SEPARATE_TRANSACTIONS { TRANSACTION_START, PUBLISH, TRANSACTION_COMMIT, TRANSACTION_START, PUBLISH, TRANSACTION_COMMIT }; + const std::vector<KafkaTestProducer::PublishEvent> NON_COMMITTED_TRANSACTION { TRANSACTION_START, PUBLISH, PUBLISH }; + const std::vector<KafkaTestProducer::PublishEvent> COMMIT_AND_CANCEL { TRANSACTION_START, PUBLISH, CANCEL }; + + const std::string KEEP_FIRST = ConsumeKafka::MSG_HEADER_KEEP_FIRST; + const std::string KEEP_LATEST = ConsumeKafka::MSG_HEADER_KEEP_LATEST; + const std::string COMMA_SEPARATED_MERGE = ConsumeKafka::MSG_HEADER_COMMA_SEPARATED_MERGE; + + static const std::string PRODUCER_TOPIC; + static const std::string TEST_MESSAGE_KEY; + + // Relationships + const core::Relationship success {"success", "description"}; + const core::Relationship failure {"failure", "description"}; + + ConsumeKafkaTest() : + logTestController_(LogTestController::getInstance()), + logger_(logging::LoggerFactory<ConsumeKafkaTest>::getLogger()) { + reInitialize(); + } + + virtual ~ConsumeKafkaTest() { + logTestController_.reset(); + } + + protected: + void reInitialize() { + testController_.reset(new TestController()); + plan_ = testController_->createPlan(); + logTestController_.setError<LogTestController>(); + logTestController_.setError<TestPlan>(); + logTestController_.setTrace<ConsumeKafka>(); + logTestController_.setTrace<ConsumeKafkaTest>(); + logTestController_.setTrace<KafkaTestProducer>(); + logTestController_.setDebug<ExtractText>(); + logTestController_.setDebug<core::ProcessContext>(); + } + + void optional_set_property(const std::shared_ptr<core::Processor>& processor, const std::string& property_name, const optional<std::string>& opt_value) { + if (opt_value) { + plan_->setProperty(processor, property_name, opt_value.value()); + } + } + + std::string decode_key(const std::string& key, const optional<std::string>& key_attribute_encoding) { + if (!key_attribute_encoding || utils::StringUtils::equalsIgnoreCase(ConsumeKafka::KEY_ATTR_ENCODING_UTF_8, key_attribute_encoding.value())) { + return key; + } + if (utils::StringUtils::equalsIgnoreCase(ConsumeKafka::ConsumeKafka::KEY_ATTR_ENCODING_HEX, key_attribute_encoding.value())) { + return utils::StringUtils::from_hex(key); + } + throw std::runtime_error("Message Header Encoding does not match any of the presets in the test."); + } + + std::vector<std::string> sort_and_split_messages(const std::vector<std::string>& messages_on_topic, const optional<std::string>& message_demarcator) { + if (message_demarcator) { + std::vector<std::string> sorted_split_messages; + for (const auto& message : messages_on_topic) { + std::vector<std::string> split_message = utils::StringUtils::split(message, message_demarcator.value()); + std::move(split_message.begin(), split_message.end(), std::back_inserter(sorted_split_messages)); + } + std::sort(sorted_split_messages.begin(), sorted_split_messages.end()); + return sorted_split_messages; + } + std::vector<std::string> sorted_messages{ messages_on_topic.cbegin(), messages_on_topic.cend() }; + std::sort(sorted_messages.begin(), sorted_messages.end()); + return sorted_messages; + } + + static const std::chrono::seconds MAX_CONSUMEKAFKA_POLL_TIME_SECONDS; + static const std::string ATTRIBUTE_FOR_CAPTURING_CONTENT; + static const std::string TEST_FILE_NAME_POSTFIX; + + std::unique_ptr<TestController> testController_; + std::shared_ptr<TestPlan> plan_; + LogTestController& logTestController_; + std::shared_ptr<logging::Logger> logger_; +}; + +class ConsumeKafkaPropertiesTest : public ConsumeKafkaTest { + public: + ConsumeKafkaPropertiesTest() : ConsumeKafkaTest() {} + virtual ~ConsumeKafkaPropertiesTest() { + logTestController_.reset(); + } + + void single_consumer_with_plain_text_test( + bool expect_config_valid, + bool expect_fixed_message_order, + const std::vector<std::pair<std::string, std::string>>& expect_header_attributes, + const std::vector<std::string>& messages_on_topic, + const std::vector<KafkaTestProducer::PublishEvent>& transaction_events, + const std::vector<std::pair<std::string, std::string>>& message_headers, + const std::string& kafka_brokers, + const std::string& security_protocol, + const std::string& topic_names, + const optional<std::string>& topic_name_format, + const optional<bool>& honor_transactions, + const optional<std::string>& group_id, + const optional<std::string>& offset_reset, + const optional<std::string>& key_attribute_encoding, + const optional<std::string>& message_demarcator, + const optional<std::string>& message_header_encoding, + const optional<std::string>& headers_to_add_as_attributes, + const optional<std::string>& duplicate_header_handling, + const optional<std::string>& max_poll_records, + const optional<std::string>& max_poll_time, + const optional<std::string>& session_timeout) { Review comment: Could these parameters be extracted to a struct and refer to them by name when initializing to be more explicit? I think it is hard currently to figure out which value represents what parameter just by looking at the function call. Default values could also be used to more easily distinguish the differences of the parameters between test cases. ########## File path: extensions/librdkafka/ConsumeKafka.cpp ########## @@ -0,0 +1,522 @@ +/** + * 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. + */ + +#include "ConsumeKafka.h" + +#include <algorithm> +#include <limits> + +#include "core/PropertyValidation.h" +#include "utils/ProcessorConfigUtils.h" +#include "utils/gsl.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace processors { + +constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS; +constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME; + +core::Property ConsumeKafka::KafkaBrokers(core::PropertyBuilder::createProperty("Kafka Brokers") + ->withDescription("A comma-separated list of known Kafka Brokers in the format <host>:<port>.") + ->withDefaultValue("localhost:9092", core::StandardValidators::get().NON_BLANK_VALIDATOR) + ->supportsExpressionLanguage(true) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::SecurityProtocol(core::PropertyBuilder::createProperty("Security Protocol") + ->withDescription("This property is currently not supported. Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.") + ->withAllowableValues<std::string>({SECURITY_PROTOCOL_PLAINTEXT/*, SECURITY_PROTOCOL_SSL, SECURITY_PROTOCOL_SASL_PLAINTEXT, SECURITY_PROTOCOL_SASL_SSL*/ }) + ->withDefaultValue(SECURITY_PROTOCOL_PLAINTEXT) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::TopicNames(core::PropertyBuilder::createProperty("Topic Names") + ->withDescription("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma separated.") + ->supportsExpressionLanguage(true) + ->build()); + +core::Property ConsumeKafka::TopicNameFormat(core::PropertyBuilder::createProperty("Topic Name Format") + ->withDescription("Specifies whether the Topic(s) provided are a comma separated list of names or a single regular expression.") + ->withAllowableValues<std::string>({TOPIC_FORMAT_NAMES, TOPIC_FORMAT_PATTERNS}) + ->withDefaultValue(TOPIC_FORMAT_NAMES) + ->build()); + +core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions") + ->withDescription( + "Specifies whether or not NiFi should honor transactional guarantees when communicating with Kafka. If false, the Processor will use an \"isolation level\" of " + "read_uncomitted. This means that messages will be received as soon as they are written to Kafka but will be pulled, even if the producer cancels the transactions. " + "If this value is true, NiFi will not receive any messages for which the producer's transaction was canceled, but this can result in some latency since the consumer " + "must wait for the producer to finish its entire transaction instead of pulling as the messages become available.") + ->withDefaultValue<bool>(true) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID") + ->withDescription("A Group ID is used to identify consumers that are within the same consumer group. Corresponds to Kafka's 'group.id' property.") + ->supportsExpressionLanguage(true) + ->build()); + +core::Property ConsumeKafka::OffsetReset(core::PropertyBuilder::createProperty("Offset Reset") + ->withDescription("Allows you to manage the condition when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that " + "data has been deleted). Corresponds to Kafka's 'auto.offset.reset' property.") + ->withAllowableValues<std::string>({OFFSET_RESET_EARLIEST, OFFSET_RESET_LATEST, OFFSET_RESET_NONE}) + ->withDefaultValue(OFFSET_RESET_LATEST) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::KeyAttributeEncoding(core::PropertyBuilder::createProperty("Key Attribute Encoding") + ->withDescription("FlowFiles that are emitted have an attribute named 'kafka.key'. This property dictates how the value of the attribute should be encoded.") + ->withAllowableValues<std::string>({KEY_ATTR_ENCODING_UTF_8, KEY_ATTR_ENCODING_HEX}) + ->withDefaultValue(KEY_ATTR_ENCODING_UTF_8) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::MessageDemarcator(core::PropertyBuilder::createProperty("Message Demarcator") + ->withDescription("Since KafkaConsumer receives messages in batches, you have an option to output FlowFiles which contains all Kafka messages in a single batch " + "for a given topic and partition and this property allows you to provide a string (interpreted as UTF-8) to use for demarcating apart multiple Kafka messages. " + "This is an optional property and if not provided each Kafka message received will result in a single FlowFile which time it is triggered. ") + ->supportsExpressionLanguage(true) + ->build()); + +core::Property ConsumeKafka::MessageHeaderEncoding(core::PropertyBuilder::createProperty("Message Header Encoding") + ->withDescription("Any message header that is found on a Kafka message will be added to the outbound FlowFile as an attribute. This property indicates the Character Encoding " + "to use for deserializing the headers.") + ->withAllowableValues<std::string>({MSG_HEADER_ENCODING_UTF_8, MSG_HEADER_ENCODING_HEX}) + ->withDefaultValue(MSG_HEADER_ENCODING_UTF_8) + ->build()); + +core::Property ConsumeKafka::HeadersToAddAsAttributes(core::PropertyBuilder::createProperty("Headers To Add As Attributes") + ->withDescription("A Regular Expression that is matched against all message headers. Any message header whose name matches the regex will be added to the FlowFile " + "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. If two messages have a different value for the same header and that " + "header is selected by the provided regex, then those two messages must be added to different FlowFiles. As a result, users should be cautious about using a " + "regex like \".*\" if messages are expected to have header values that are unique per message, such as an identifier or timestamp, because it will prevent MiNiFi " + "from bundling the messages together efficiently.") + ->build()); + +core::Property ConsumeKafka::DuplicateHeaderHandling(core::PropertyBuilder::createProperty("Duplicate Header Handling") + ->withDescription("For headers to be added as attributes, this option specifies how to handle cases where multiple headers are present with the same key. " + "For example in case of receiving these two headers: \"Accept: text/html\" and \"Accept: application/xml\" and we want to attach the value of \"Accept\" " + "as a FlowFile attribute:\n" + " - \"Keep First\" attaches: \"Accept -> text/html\"\n" + " - \"Keep Latest\" attaches: \"Accept -> application/xml\"\n" + " - \"Comma-separated Merge\" attaches: \"Accept -> text/html, application/xml\"\n") + ->withAllowableValues<std::string>({MSG_HEADER_KEEP_FIRST, MSG_HEADER_KEEP_LATEST, MSG_HEADER_COMMA_SEPARATED_MERGE}) + ->withDefaultValue(MSG_HEADER_KEEP_LATEST) // Mirroring NiFi behaviour + ->build()); + +core::Property ConsumeKafka::MaxPollRecords(core::PropertyBuilder::createProperty("Max Poll Records") + ->withDescription("Specifies the maximum number of records Kafka should return when polling each time the processor is triggered.") + ->withDefaultValue<unsigned int>(DEFAULT_MAX_POLL_RECORDS) + ->build()); + +core::Property ConsumeKafka::MaxPollTime(core::PropertyBuilder::createProperty("Max Poll Time") + ->withDescription("Specifies the maximum amount of time the consumer can use for polling data from the brokers. " + "Polling is a blocking operation, so the upper limit of this value is specified in 4 seconds.") + ->withDefaultValue(DEFAULT_MAX_POLL_TIME, std::make_shared<core::ConsumeKafkaMaxPollTimeValidator>(std::string("ConsumeKafkaMaxPollTimeValidator"))) + ->isRequired(true) + ->build()); + +core::Property ConsumeKafka::SessionTimeout(core::PropertyBuilder::createProperty("Session Timeout") + ->withDescription("Client group session and failure detection timeout. The consumer sends periodic heartbeats " + "to indicate its liveness to the broker. If no hearts are received by the broker for a group member within " + "the session timeout, the broker will remove the consumer from the group and trigger a rebalance. " + "The allowed range is configured with the broker configuration properties group.min.session.timeout.ms and group.max.session.timeout.ms.") + ->withDefaultValue<core::TimePeriodValue>("60 seconds") + ->build()); + +const core::Relationship ConsumeKafka::Success("success", "Incoming kafka messages as flowfiles. Depending on the demarcation strategy, this can be one or multiple flowfiles per message."); + +void ConsumeKafka::initialize() { + setSupportedProperties({ + KafkaBrokers, + SecurityProtocol, + TopicNames, + TopicNameFormat, + HonorTransactions, + GroupID, + OffsetReset, + KeyAttributeEncoding, + MessageDemarcator, + MessageHeaderEncoding, + HeadersToAddAsAttributes, + DuplicateHeaderHandling, + MaxPollRecords, + MaxPollTime, + SessionTimeout + }); + setSupportedRelationships({ + Success, + }); +} + +void ConsumeKafka::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /* sessionFactory */) { + // Required properties + kafka_brokers_ = utils::getRequiredPropertyOrThrow(context, KafkaBrokers.getName()); + security_protocol_ = utils::getRequiredPropertyOrThrow(context, SecurityProtocol.getName()); + topic_names_ = utils::listFromRequiredCommaSeparatedProperty(context, TopicNames.getName()); + topic_name_format_ = utils::getRequiredPropertyOrThrow(context, TopicNameFormat.getName()); + honor_transactions_ = utils::parseBooleanPropertyOrThrow(context, HonorTransactions.getName()); + group_id_ = utils::getRequiredPropertyOrThrow(context, GroupID.getName()); + offset_reset_ = utils::getRequiredPropertyOrThrow(context, OffsetReset.getName()); + key_attribute_encoding_ = utils::getRequiredPropertyOrThrow(context, KeyAttributeEncoding.getName()); + max_poll_time_milliseconds_ = utils::parseTimePropertyMSOrThrow(context, MaxPollTime.getName()); + session_timeout_milliseconds_ = utils::parseTimePropertyMSOrThrow(context, SessionTimeout.getName()); + + // Optional properties + context->getProperty(MessageDemarcator.getName(), message_demarcator_); + context->getProperty(MessageHeaderEncoding.getName(), message_header_encoding_); + context->getProperty(DuplicateHeaderHandling.getName(), duplicate_header_handling_); + + headers_to_add_as_attributes_ = utils::listFromCommaSeparatedProperty(context, HeadersToAddAsAttributes.getName()); + max_poll_records_ = gsl::narrow<std::size_t>(utils::getOptionalUintProperty(context, MaxPollRecords.getName()).value_or(DEFAULT_MAX_POLL_RECORDS)); + + // For now security protocols are not yet supported + if (!utils::StringUtils::equalsIgnoreCase(SECURITY_PROTOCOL_PLAINTEXT, security_protocol_)) { + throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Security protocols are not supported yet."); + } + + if (!utils::StringUtils::equalsIgnoreCase(KEY_ATTR_ENCODING_UTF_8, key_attribute_encoding_) && !utils::StringUtils::equalsIgnoreCase(KEY_ATTR_ENCODING_HEX, key_attribute_encoding_)) { + throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Unsupported key attribute encoding: " + key_attribute_encoding_); + } + + if (!utils::StringUtils::equalsIgnoreCase(MSG_HEADER_ENCODING_UTF_8, message_header_encoding_) && !utils::StringUtils::equalsIgnoreCase(MSG_HEADER_ENCODING_HEX, message_header_encoding_)) { + throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Unsupported message header encoding: " + key_attribute_encoding_); + } + + configure_new_connection(context); +} + +void rebalance_cb(rd_kafka_t* rk, rd_kafka_resp_err_t trigger, rd_kafka_topic_partition_list_t* partitions, void* /*opaque*/) { + // Cooperative, incremental assignment is not supported in the current librdkafka version + std::shared_ptr<logging::Logger> logger{logging::LoggerFactory<ConsumeKafka>::getLogger()}; + logger->log_debug("Rebalance triggered."); + rd_kafka_resp_err_t assign_error = RD_KAFKA_RESP_ERR_NO_ERROR; + switch (trigger) { + case RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS: + logger->log_debug("assigned"); + utils::print_topics_list(logger, partitions); + assign_error = rd_kafka_assign(rk, partitions); + break; + + case RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS: + logger->log_debug("revoked:"); + rd_kafka_commit(rk, partitions, /* async = */ 0); // Sync commit, maybe unneccessary + utils::print_topics_list(logger, partitions); + assign_error = rd_kafka_assign(rk, NULL); + break; + + default: + logger->log_debug("failed: %s", rd_kafka_err2str(trigger)); + assign_error = rd_kafka_assign(rk, NULL); + break; + } + logger->log_debug("assign failure: %s", rd_kafka_err2str(assign_error)); +} + +void ConsumeKafka::createTopicPartitionList() { + kf_topic_partition_list_ = { rd_kafka_topic_partition_list_new(topic_names_.size()), utils::rd_kafka_topic_partition_list_deleter() }; + + // On subscriptions any topics prefixed with ^ will be regex matched + if (topic_name_format_ == "pattern") { + for (const std::string& topic : topic_names_) { + const std::string regex_format = "^" + topic; + rd_kafka_topic_partition_list_add(kf_topic_partition_list_.get(), regex_format.c_str(), RD_KAFKA_PARTITION_UA); + } + } else { + for (const std::string& topic : topic_names_) { + rd_kafka_topic_partition_list_add(kf_topic_partition_list_.get(), topic.c_str(), RD_KAFKA_PARTITION_UA); + } + } + + // Subscribe to topic set using balanced consumer groups + // Subscribing from the same process without an inbetween unsubscribe call + // Does not seem to be triggering a rebalance (maybe librdkafka bug?) + // This might happen until the cross-overship between processors and connections are settled + rd_kafka_resp_err_t subscribe_response = rd_kafka_subscribe(consumer_.get(), kf_topic_partition_list_.get()); + if (RD_KAFKA_RESP_ERR_NO_ERROR != subscribe_response) { + logger_->log_error("rd_kafka_subscribe error %d: %s", subscribe_response, rd_kafka_err2str(subscribe_response)); + } +} + +void ConsumeKafka::extend_config_from_dynamic_properties(const core::ProcessContext* context) { + using utils::setKafkaConfigurationField; + + const std::vector<std::string> dynamic_prop_keys = context->getDynamicPropertyKeys(); + if (dynamic_prop_keys.empty()) { + return; + } + logger_->log_info("Loading %d extra kafka configuration fields from ConsumeKafka dynamic properties:", dynamic_prop_keys.size()); + for (const std::string& key : dynamic_prop_keys) { + std::string value; + gsl_Expects(context->getDynamicProperty(key, value)); + logger_->log_info("%s: %s", key.c_str(), value.c_str()); + setKafkaConfigurationField(conf_.get(), key, value); + } +} + +void ConsumeKafka::configure_new_connection(const core::ProcessContext* context) { + using utils::setKafkaConfigurationField; + + conf_ = { rd_kafka_conf_new(), utils::rd_kafka_conf_deleter() }; + if (conf_ == nullptr) { + throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create rd_kafka_conf_t object"); + } + + // Set rebalance callback for use with coordinated consumer group balancing + // Rebalance handlers are needed for the initial configuration of the consumer + // If they are not set, offset reset is ignored and polling produces messages + // Registering a rebalance_cb turns off librdkafka's automatic partition assignment/revocation and instead delegates that responsibility to the application's rebalance_cb. + rd_kafka_conf_set_rebalance_cb(conf_.get(), rebalance_cb); + + // Uncomment this for librdkafka debug logs: + // setKafkaConfigurationField(conf_.get(), "debug", "all"); + + setKafkaConfigurationField(conf_.get(), "bootstrap.servers", kafka_brokers_); + setKafkaConfigurationField(conf_.get(), "auto.offset.reset", "latest"); + setKafkaConfigurationField(conf_.get(), "enable.auto.commit", "false"); + setKafkaConfigurationField(conf_.get(), "enable.auto.offset.store", "false"); + setKafkaConfigurationField(conf_.get(), "isolation.level", honor_transactions_ ? "read_committed" : "read_uncommitted"); + setKafkaConfigurationField(conf_.get(), "group.id", group_id_); + setKafkaConfigurationField(conf_.get(), "session.timeout.ms", std::to_string(session_timeout_milliseconds_.count())); + setKafkaConfigurationField(conf_.get(), "max.poll.interval.ms", "600000"); // Twice the default, arbitrarily chosen + + // This is a librdkafka option, but the communication timeout is also specified in each of the + // relevant API calls. Could be redundant, but it probably does not hurt to set this + setKafkaConfigurationField(conf_.get(), "metadata.request.timeout.ms", std::to_string(METADATA_COMMUNICATIONS_TIMEOUT_MS)); + + extend_config_from_dynamic_properties(context); + + std::array<char, 512U> errstr{}; + consumer_ = { rd_kafka_new(RD_KAFKA_CONSUMER, conf_.release(), errstr.data(), errstr.size()), utils::rd_kafka_consumer_deleter() }; + if (consumer_ == nullptr) { + const std::string error_msg { errstr.begin(), errstr.end() }; + throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer %s" + error_msg); + } + + createTopicPartitionList(); + + // Changing the partition list should happen only as part as the initialization of offsets + // a function like `rd_kafka_position()` might have unexpected effects + // for instance when a consumer gets assigned a partition it used to + // consume at an earlier rebalance. + // + // As far as I understand, instead of rd_kafka_position() an rd_kafka_committed() call if preferred here, + // as it properly fetches offsets from the broker + if (RD_KAFKA_RESP_ERR_NO_ERROR != rd_kafka_committed(consumer_.get(), kf_topic_partition_list_.get(), METADATA_COMMUNICATIONS_TIMEOUT_MS)) { + logger_ -> log_error("Retrieving committed offsets for topics+partitions failed."); + } + + rd_kafka_resp_err_t poll_set_consumer_response = rd_kafka_poll_set_consumer(consumer_.get()); + if (RD_KAFKA_RESP_ERR_NO_ERROR != poll_set_consumer_response) { + logger_->log_error("rd_kafka_poll_set_consumer error %d: %s", poll_set_consumer_response, rd_kafka_err2str(poll_set_consumer_response)); + } + + // There is no rd_kafka_seek alternative for rd_kafka_topic_partition_list_t, only rd_kafka_topic_t + // rd_kafka_topic_partition_list_set_offset should reset the offsets to the latest (or whatever is set in the config), + // Also, rd_kafka_committed should also fetch and set latest the latest offset + // In reality, neither of them seem to work (not even with calling rd_kafka_position()) + logger_->log_info("Resetting offset manually."); + while (true) { + std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter> + message_wrapper{ rd_kafka_consumer_poll(consumer_.get(), max_poll_time_milliseconds_.count()), utils::rd_kafka_message_deleter() }; + + if (!message_wrapper || RD_KAFKA_RESP_ERR_NO_ERROR != message_wrapper->err) { + break; + } + utils::print_kafka_message(message_wrapper.get(), logger_); + // Commit offsets on broker for the provided list of partitions + logger_->log_info("Committing offset: %" PRId64 ".", message_wrapper->offset); + rd_kafka_commit_message(consumer_.get(), message_wrapper.get(), /* async = */ 0); + } + logger_->log_info("Done resetting offset manually."); +} + +std::string ConsumeKafka::extract_message(const rd_kafka_message_t* rkmessage) { + if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) { + throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker."); + } + return { reinterpret_cast<char*>(rkmessage->payload), rkmessage->len }; +} + +std::vector<std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>> ConsumeKafka::poll_kafka_messages() { + std::vector<std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>> messages; + messages.reserve(max_poll_records_); + const auto start = std::chrono::high_resolution_clock::now(); + auto elapsed = std::chrono::high_resolution_clock::now() - start; + while (messages.size() < max_poll_records_ && elapsed < max_poll_time_milliseconds_) { + logger_-> log_debug("Polling for new messages for %d milliseconds...", max_poll_time_milliseconds_.count()); + rd_kafka_message_t* message = rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()); + if (!message || RD_KAFKA_RESP_ERR_NO_ERROR != message->err) { + break; + } + utils::print_kafka_message(message, logger_); + messages.emplace_back(std::move(message), utils::rd_kafka_message_deleter()); + elapsed = std::chrono::high_resolution_clock::now() - start; + } + return messages; +} + +utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() { + if (utils::StringUtils::equalsIgnoreCase(key_attribute_encoding_, KEY_ATTR_ENCODING_UTF_8)) { + return utils::KafkaEncoding::UTF8; + } + if (utils::StringUtils::equalsIgnoreCase(key_attribute_encoding_, KEY_ATTR_ENCODING_HEX)) { + return utils::KafkaEncoding::HEX; + } + throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "\"Key Attribute Encoding\" property not recognized."); +} + +utils::KafkaEncoding ConsumeKafka::message_header_encoding_attr_to_enum() { + if (utils::StringUtils::equalsIgnoreCase(message_header_encoding_, MSG_HEADER_ENCODING_UTF_8)) { + return utils::KafkaEncoding::UTF8; + } + if (utils::StringUtils::equalsIgnoreCase(message_header_encoding_, MSG_HEADER_ENCODING_HEX)) { + return utils::KafkaEncoding::HEX; + } + throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "Key Attribute Encoding property not recognized."); +} + +std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) { + if (MSG_HEADER_KEEP_FIRST == duplicate_header_handling_) { + return matching_headers.front(); + } + if (MSG_HEADER_KEEP_LATEST == duplicate_header_handling_) { + return matching_headers.back(); + } + if (MSG_HEADER_COMMA_SEPARATED_MERGE == duplicate_header_handling_) { + return utils::StringUtils::join(", ", matching_headers); + } + throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "\"Duplicate Header Handling\" property not recognized."); +} + +std::vector<std::string> ConsumeKafka::get_matching_headers(const rd_kafka_message_t* message, const std::string& header_name) { + // Headers fetched this way are freed when rd_kafka_message_destroy is called + // Detaching them using rd_kafka_message_detach_headers does not seem to work + rd_kafka_headers_t* headers_raw; + const rd_kafka_resp_err_t get_header_response = rd_kafka_message_headers(message, &headers_raw); + if (RD_KAFKA_RESP_ERR__NOENT == get_header_response) { + return {}; + } + if (RD_KAFKA_RESP_ERR_NO_ERROR != get_header_response) { + logger_->log_error("Failed to fetch message headers: %d: %s", rd_kafka_last_error(), rd_kafka_err2str(rd_kafka_last_error())); + } + std::vector<std::string> matching_headers; + for (std::size_t header_idx = 0;; ++header_idx) { + const char* value; // Not to be freed + std::size_t size; + if (RD_KAFKA_RESP_ERR_NO_ERROR != rd_kafka_header_get(headers_raw, header_idx, header_name.c_str(), (const void**)(&value), &size)) { + break; + } + if (size < std::numeric_limits<int>::max()) { + logger_->log_debug("%.*s", static_cast<int>(size), value); + } + matching_headers.emplace_back(value, size); + } + return matching_headers; +} + +std::vector<std::pair<std::string, std::string>> ConsumeKafka::get_flowfile_attributes_from_message_header(const rd_kafka_message_t* message) { + std::vector<std::pair<std::string, std::string>> attributes_from_headers; + for (const std::string& header_name : headers_to_add_as_attributes_) { + const std::vector<std::string> matching_headers = get_matching_headers(message, header_name); + if (matching_headers.size()) { + attributes_from_headers.emplace_back(header_name, utils::get_encoded_string(resolve_duplicate_headers(matching_headers), message_header_encoding_attr_to_enum())); + } + } + return attributes_from_headers; +} + +class WriteCallback : public OutputStreamCallback { + public: + WriteCallback(char *data, uint64_t size) : + data_(reinterpret_cast<uint8_t*>(data)), + dataSize_(size) {} + uint8_t* data_; + uint64_t dataSize_; + int64_t process(const std::shared_ptr<io::BaseStream>& stream) { + int64_t ret = 0; + if (data_ && dataSize_ > 0) + ret = stream->write(data_, dataSize_); + return ret; + } +}; + +std::vector<std::shared_ptr<FlowFileRecord>> ConsumeKafka::transform_messages_into_flowfiles( + const std::vector<std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>>& messages, core::ProcessSession* session) { + std::vector<std::shared_ptr<FlowFileRecord>> flow_files_created; + for (const auto& message : messages) { + std::string message_content = extract_message(message.get()); + if (message_content.empty()) { + logger_->log_debug("Error: message received contains no data."); + return {}; + } + + std::vector<std::pair<std::string, std::string>> attributes_from_headers = get_flowfile_attributes_from_message_header(message.get()); + std::vector<std::string> split_message{ utils::StringUtils::split(message_content, message_demarcator_) }; + for (auto& flowfile_content : split_message) { + std::shared_ptr<FlowFileRecord> flow_file = std::static_pointer_cast<FlowFileRecord>(session->create()); + if (flow_file == nullptr) { + return {}; + } + // flowfile content is consumed here + WriteCallback stream_writer_callback(&flowfile_content[0], flowfile_content.size()); + session->write(flow_file, &stream_writer_callback); + for (const auto& kv : attributes_from_headers) { + flow_file->setAttribute(kv.first, kv.second); + } + // We do not currently support batching messages into a single flowfile + flow_file->setAttribute(KAFKA_COUNT_ATTR, "1"); + const utils::optional<std::string> message_key = utils::get_encoded_message_key(message.get(), key_attr_encoding_attr_to_enum()); + if (message_key) { + flow_file->setAttribute(KAFKA_MESSAGE_KEY_ATTR, message_key.value()); + } + flow_file->setAttribute(KAFKA_OFFSET_ATTR, std::to_string(message->offset)); + flow_file->setAttribute(KAFKA_PARTITION_ATTR, std::to_string(message->partition)); + flow_file->setAttribute(KAFKA_TOPIC_ATTR, rd_kafka_topic_name(message->rkt)); + flow_files_created.emplace_back(std::move(flow_file)); Review comment: This process could be extracted to a separate function just for more clarity. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
